diff --git a/backends-clickhouse/pom.xml b/backends-clickhouse/pom.xml index 94df2f36f183..a37734b150bb 100644 --- a/backends-clickhouse/pom.xml +++ b/backends-clickhouse/pom.xml @@ -270,6 +270,12 @@ src/main/delta-${delta.binary.version}/**/*.scala src/test/delta-${delta.binary.version}/**/*.scala + + src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/commands/*.scala + src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/files/*.scala + src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/DeltaLog.scala + src/main/delta-${delta.binary.version}/org/apache/spark/sql/delta/Snapshot.scala + diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaLog.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaLog.scala index 0f6455997e56..57c6c8550fc0 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaLog.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/DeltaLog.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.storage.LogStoreProvider +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} import org.apache.spark.sql.types.{StructField, StructType} @@ -214,7 +215,9 @@ class DeltaLog private ( */ def startTransaction(): OptimisticTransaction = { update() + // --- modified start new ClickhouseOptimisticTransaction(this, None) + // --- modified end } /** @@ -443,7 +446,13 @@ class DeltaLog private ( val fileIndex = TahoeLogFileIndex(spark, this, dataPath, snapshotToUse, partitionFilters, isTimeTravelQuery) - val bucketSpec: Option[BucketSpec] = ClickHouseTableV2.getTable(this).bucketOption + // --- modified start + val bucketSpec: Option[BucketSpec] = + if (ClickHouseConfig.isMergeTreeFormatEngine(snapshotToUse.metadata.configuration)) { + ClickHouseTableV2.getTable(this).bucketOption + } else { + None + } new DeltaHadoopFsRelation( fileIndex, partitionSchema = @@ -464,20 +473,28 @@ class DeltaLog private ( spark, this ) + // --- modified end } - override def fileFormat(metadata: Metadata = metadata): FileFormat = - ClickHouseTableV2.getTable(this).getFileFormat(metadata) - + override def fileFormat(metadata: Metadata = metadata): FileFormat = { + // --- modified start + if (ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { + ClickHouseTableV2.getTable(this).getFileFormat(metadata) + } else { + super.fileFormat(metadata) + } + // --- modified end + } } object DeltaLog extends DeltaLogging { + // --- modified start @SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass")) private class DeltaHadoopFsRelation( location: FileIndex, partitionSchema: StructType, - // The top-level columns in `dataSchema` should match the actual physical file schema, otherwise - // the ORC data source may not work with the by-ordinal mode. + // The top-level columns in `dataSchema` should match the actual physical file schema, + // otherwise the ORC data source may not work with the by-ordinal mode. dataSchema: StructType, bucketSpec: Option[BucketSpec], fileFormat: FileFormat, @@ -502,6 +519,7 @@ object DeltaLog extends DeltaLogging { ).run(sparkSession) } } + // --- modified end /** * The key type of `DeltaLog` cache. It's a pair of the canonicalized table path and the file diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/Snapshot.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/Snapshot.scala index 2233aa0cd1ef..2e4d6bb2207c 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/Snapshot.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/Snapshot.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.stats.{DataSkippingReader, DeltaScan, FileSizeHistogram, StatisticsCollection} import org.apache.spark.sql.delta.util.StateCache import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -404,6 +405,7 @@ class Snapshot( s"${getClass.getSimpleName}(path=$path, version=$version, metadata=$metadata, " + s"logSegment=$logSegment, checksumOpt=$checksumOpt)" + // --- modified start override def filesForScan( projection: Seq[Attribute], filters: Seq[Expression], @@ -418,31 +420,36 @@ class Snapshot( } private def replaceWithAddMergeTreeParts(deltaScan: DeltaScan) = { - DeltaScan.apply( - deltaScan.version, - deltaScan.files - .map( - addFile => { - val addFileAsKey = AddFileAsKey(addFile) - - val ret = ClickhouseSnapshot.addFileToAddMTPCache.get(addFileAsKey) - // this is for later use - ClickhouseSnapshot.pathToAddMTPCache.put(ret.fullPartPath(), ret) - ret - }), - deltaScan.total, - deltaScan.partition, - deltaScan.scanned - )( - deltaScan.scannedSnapshot, - deltaScan.partitionFilters, - deltaScan.dataFilters, - deltaScan.unusedFilters, - deltaScan.projection, - deltaScan.scanDurationMs, - deltaScan.dataSkippingType - ) + if (ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { + DeltaScan.apply( + deltaScan.version, + deltaScan.files + .map( + addFile => { + val addFileAsKey = AddFileAsKey(addFile) + + val ret = ClickhouseSnapshot.addFileToAddMTPCache.get(addFileAsKey) + // this is for later use + ClickhouseSnapshot.pathToAddMTPCache.put(ret.fullPartPath(), ret) + ret + }), + deltaScan.total, + deltaScan.partition, + deltaScan.scanned + )( + deltaScan.scannedSnapshot, + deltaScan.partitionFilters, + deltaScan.dataFilters, + deltaScan.unusedFilters, + deltaScan.projection, + deltaScan.scanDurationMs, + deltaScan.dataSkippingType + ) + } else { + deltaScan + } } + // --- modified end logInfo(s"Created snapshot $this") init() diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/DeleteCommand.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/DeleteCommand.scala index 527b9619eb5d..61b0330723f5 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/DeleteCommand.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/DeleteCommand.scala @@ -216,6 +216,7 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt if (candidateFiles.isEmpty) { Array.empty[String] } else { + // --- modified start data .filter(new Column(cond)) .select(input_file_name().as("input_files")) @@ -224,6 +225,7 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt .distinct() .as[String] .collect() + // --- modified end } } diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala index 89208dd45314..4b26d5d5d949 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala @@ -407,7 +407,9 @@ case class MergeIntoCommand( val recordTouchedFileName = udf { (fileName: String) => { + // --- modified start fileName.split(",").foreach(name => touchedFilesAccum.add(name)) + // --- modified end 1 } }.asNondeterministic() diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala index 571ac63f1eb6..6437ab75903b 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.skipping.MultiDimClustering import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.execution.command.{LeafRunnableCommand, RunnableCommand} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata.AddMergeTreeParts import org.apache.spark.sql.execution.datasources.v2.clickhouse.utils.CHDataSourceUtils import org.apache.spark.sql.execution.metric.SQLMetric @@ -127,8 +128,10 @@ case class OptimizeTableCommand( override val otherCopyArgs: Seq[AnyRef] = zOrderBy :: Nil override def run(sparkSession: SparkSession): Seq[Row] = { + // --- modified start CHDataSourceUtils.ensureClickHouseTableV2(tableId, sparkSession) val deltaLog = getDeltaLogClickhouse(sparkSession, path, tableId, "OPTIMIZE") + // --- modified end val partitionColumns = deltaLog.snapshot.metadata.partitionColumns // Parse the predicate expression into Catalyst expression and verify only simple filters @@ -177,6 +180,10 @@ class OptimizeExecutor( def optimize(): Seq[Row] = { recordDeltaOperation(deltaLog, "delta.optimize") { + // --- modified start + val isMergeTreeFormat = ClickHouseConfig + .isMergeTreeFormatEngine(deltaLog.snapshot.metadata.configuration) + // --- modified end val minFileSize = sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE) val maxFileSize = @@ -194,37 +201,59 @@ class OptimizeExecutor( // select all files in case of multi-dimensional clustering val filesToProcess = candidateFiles.filter(_.size < minFileSize || isMultiDimClustering) - val partitionsToCompact = filesToProcess - .groupBy(file => (file.asInstanceOf[AddMergeTreeParts].bucketNum, file.partitionValues)) - .toSeq - - val jobs = groupFilesIntoBinsClickhouse(partitionsToCompact, maxFileSize) - - val parallelJobCollection = new ParVector(jobs.toVector) + // --- modified start // Create a task pool to parallelize the submission of optimization jobs to Spark. val threadPool = ThreadUtils.newForkJoinPool( "OptimizeJob", sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_THREADS)) + val (updates, jobs) = if (isMergeTreeFormat) { + val partitionsToCompact = filesToProcess + .groupBy(file => (file.asInstanceOf[AddMergeTreeParts].bucketNum, file.partitionValues)) + .toSeq + + val jobs = groupFilesIntoBinsClickhouse(partitionsToCompact, maxFileSize) + + val parallelJobCollection = new ParVector(jobs.toVector) + + val updates = + try { + val forkJoinPoolTaskSupport = new ForkJoinTaskSupport(threadPool) + parallelJobCollection.tasksupport = forkJoinPoolTaskSupport + + parallelJobCollection + .flatMap( + partitionBinGroup => + runOptimizeBinJobClickhouse( + txn, + partitionBinGroup._1._2, + partitionBinGroup._1._1, + partitionBinGroup._2, + maxFileSize)) + .seq + } finally { + threadPool.shutdownNow() + } + (updates, jobs) + } else { + val partitionsToCompact = filesToProcess.groupBy(_.partitionValues).toSeq + + val jobs = groupFilesIntoBins(partitionsToCompact, maxFileSize) - val updates = - try { + val parallelJobCollection = new ParVector(jobs.toVector) + + val updates = try { val forkJoinPoolTaskSupport = new ForkJoinTaskSupport(threadPool) parallelJobCollection.tasksupport = forkJoinPoolTaskSupport - parallelJobCollection - .flatMap( - partitionBinGroup => - runOptimizeBinJobClickhouse( - txn, - partitionBinGroup._1._2, - partitionBinGroup._1._1, - partitionBinGroup._2, - maxFileSize)) - .seq + parallelJobCollection.flatMap(partitionBinGroup => + runOptimizeBinJob(txn, partitionBinGroup._1, partitionBinGroup._2, maxFileSize)).seq } finally { threadPool.shutdownNow() } + (updates, jobs) + } + // --- modified end val addedFiles = updates.collect { case a: AddFile => a } val removedFiles = updates.collect { case r: RemoveFile => r } diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/UpdateCommand.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/UpdateCommand.scala index f6e2968b703f..42a081788eb1 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/UpdateCommand.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/UpdateCommand.scala @@ -144,6 +144,7 @@ case class UpdateCommand( }.asNondeterministic() val pathsToRewrite = withStatusCode("DELTA", UpdateCommand.FINDING_TOUCHED_FILES_MSG) { + // --- modified start data .filter(new Column(updateCondition)) .filter(updatedRowUdf()) @@ -152,6 +153,7 @@ case class UpdateCommand( .distinct() .as[String] .collect() + // --- modified end } scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/VacuumCommand.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/VacuumCommand.scala index 11f6aa977904..3a390f64d559 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/VacuumCommand.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/commands/VacuumCommand.scala @@ -21,6 +21,7 @@ import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions.{FileAction, RemoveFile} import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.util.DeltaFileOperations +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.functions.{col, expr, when} import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock} @@ -41,7 +42,9 @@ import java.util.concurrent.TimeUnit */ object VacuumCommand extends VacuumCommandImpl with Serializable { - case class FileNameAndSize(path: String, length: Long, isDir: Boolean) + // --- modified start + case class FileNameAndSize(path: String, length: Long, isDir: Boolean = false) + // --- modified end /** * Additional check on retention duration to prevent people from shooting themselves in the foot. @@ -112,6 +115,11 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { "No state defined for this table. Is this really " + "a Delta table? Refusing to garbage collect.") + // --- modified start + val isMergeTreeFormat = ClickHouseConfig + .isMergeTreeFormatEngine(deltaLog.snapshot.metadata.configuration) + // --- modified end + val retentionMillis = retentionHours.map(h => TimeUnit.HOURS.toMillis(math.round(h))) checkRetentionPeriodSafety(spark, retentionMillis, deltaLog.tombstoneRetentionMillis) @@ -209,60 +217,92 @@ object VacuumCommand extends VacuumCommandImpl with Serializable { // 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 these - val diff_temp = allFilesAndDirs - .where('modificationTime < deleteBeforeTimestamp || 'isDir) - .mapPartitions { - fileStatusIterator => - val reservoirBase = new Path(basePath) - val fs = reservoirBase.getFileSystem(hadoopConf.value.value) - fileStatusIterator.flatMap { - fileStatus => - if (fileStatus.isDir) { - implicit val fileNameAndSizeEncoder = - org.apache.spark.sql.Encoders.product[FileNameAndSize] - Iterator.single( - FileNameAndSize( - relativize(fileStatus.getPath, fs, reservoirBase, isDir = true), - 0, - true) - ) - } else { - val dirs = getAllSubdirs(basePath, fileStatus.path, fs) - val dirsWithSlash = dirs.map { - p => + // --- modified start + val diff = if (isMergeTreeFormat) { + val diff_temp = allFilesAndDirs + .where('modificationTime < deleteBeforeTimestamp || 'isDir) + .mapPartitions { + fileStatusIterator => + val reservoirBase = new Path(basePath) + val fs = reservoirBase.getFileSystem(hadoopConf.value.value) + fileStatusIterator.flatMap { + fileStatus => + if (fileStatus.isDir) { + implicit val fileNameAndSizeEncoder = + org.apache.spark.sql.Encoders.product[FileNameAndSize] + Iterator.single( FileNameAndSize( - relativize(new Path(p), fs, reservoirBase, isDir = true), + relativize(fileStatus.getPath, fs, reservoirBase, isDir = true), 0, true) + ) + } else { + val dirs = getAllSubdirs(basePath, fileStatus.path, fs) + val dirsWithSlash = dirs.map { + p => + FileNameAndSize( + relativize(new Path(p), fs, reservoirBase, isDir = true), + 0, + true) + } + dirsWithSlash ++ Iterator( + FileNameAndSize( + relativize(new Path(fileStatus.path), fs, reservoirBase, isDir = false), + 0, + false)) } - dirsWithSlash ++ Iterator( - FileNameAndSize( - relativize(new Path(fileStatus.path), fs, reservoirBase, isDir = false), - 0, - false)) + } + } + .withColumn( + "dir", + when(col("isDir"), col("path")) + .otherwise(expr("substring_index(path, '/',size(split(path, '/')) -1)"))) + .groupBy(col("path"), col("dir")) + .count() + + diff_temp + .join(validFiles, diff_temp("dir") === validFiles("path"), "leftanti") + .where('count === 1) + .select('path) + .as[String] + .map { + relativePath => + assert( + !stringToPath(relativePath).isAbsolute, + "Shouldn't have any absolute paths for deletion here.") + pathToString(DeltaFileOperations.absolutePath(basePath, relativePath)) + } + } else { + allFilesAndDirs + .where('modificationTime < deleteBeforeTimestamp || 'isDir) + .mapPartitions { fileStatusIterator => + val reservoirBase = new Path(basePath) + val fs = reservoirBase.getFileSystem(hadoopConf.value.value) + fileStatusIterator.flatMap { fileStatus => + if (fileStatus.isDir) { + Iterator.single(relativize(fileStatus.getPath, fs, reservoirBase, isDir = true)) + } else { + val dirs = getAllSubdirs(basePath, fileStatus.path, fs) + val dirsWithSlash = dirs.map { p => + relativize(new Path(p), fs, reservoirBase, isDir = true) } + dirsWithSlash ++ Iterator( + relativize(new Path(fileStatus.path), fs, reservoirBase, isDir = false)) + } } - } -// .groupBy(col("path")) - .withColumn( - "dir", - when(col("isDir"), col("path")) - .otherwise(expr("substring_index(path, '/',size(split(path, '/')) -1)"))) - .groupBy(col("path"), col("dir")) - .count() - - val diff = diff_temp - .join(validFiles, diff_temp("dir") === validFiles("path"), "leftanti") - .where('count === 1) - .select('path) - .as[String] - .map { - relativePath => - assert( - !stringToPath(relativePath).isAbsolute, + }.groupBy($"value" as 'path) + .count() + .join(validFiles, Seq("path"), "leftanti") + .where('count === 1) + .select('path) + .as[String] + .map { relativePath => + assert(!stringToPath(relativePath).isAbsolute, "Shouldn't have any absolute paths for deletion here.") pathToString(DeltaFileOperations.absolutePath(basePath, relativePath)) - } + } + } + // --- modified end if (dryRun) { val numFiles = diff.count() diff --git a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala index bfc2555e82a2..4b6d56644b9e 100644 --- a/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala +++ b/backends-clickhouse/src/main/delta-20/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala @@ -212,19 +212,9 @@ class MergeTreeCommitProtocol(jobId: String, path: String, randomPrefixLength: O } override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { - if (addedFiles.nonEmpty) { - /* val fs = new Path(path, addedFiles.head._2).getFileSystem(taskContext.getConfiguration) - val statuses: Seq[FileAction] = addedFiles.map { f => - val filePath = new Path(path, new Path(new URI(f._2))) - val stat = fs.getFileStatus(filePath) - - buildActionFromAddedFile(f, stat, taskContext) - }.toSeq */ - - new TaskCommitMessage(Nil) - } else { - new TaskCommitMessage(Nil) - } + // will return TaskCommitMessage(Nil) directly, + // the FileStatus list will be get from the CH backend. + new TaskCommitMessage(Nil) } override def abortTask(taskContext: TaskAttemptContext): Unit = { diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/VacuumCommand.scala b/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/VacuumCommand.scala deleted file mode 100644 index dc833c2b2fae..000000000000 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/VacuumCommand.scala +++ /dev/null @@ -1,464 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.delta.commands - -import org.apache.spark.broadcast.Broadcast -import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession} -import org.apache.spark.sql.delta._ -import org.apache.spark.sql.delta.actions.{FileAction, RemoveFile} -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.util.DeltaFileOperations -import org.apache.spark.sql.delta.util.DeltaFileOperations.tryDeleteNonRecursive -import org.apache.spark.sql.functions._ -import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock} - -// scalastyle:off import.ordering.noEmptyLine -import com.fasterxml.jackson.databind.annotation.JsonDeserialize -import org.apache.hadoop.fs.{FileSystem, Path} - -import java.net.URI -import java.util.Date -import java.util.concurrent.TimeUnit - -import scala.collection.JavaConverters._ - -/** - * Gluten overwrite Delta: - * - * This file is copied from Delta 2.2.0. It is modified to overcome the following issues: - * 1. In Gluten, part is a directory, but VacuumCommand assumes part is a file. So we need some - * modifications to make it work. - */ - -/** - * Vacuums the table by clearing all untracked files and folders within this table. First lists all - * the files and directories in the table, and gets the relative paths with respect to the base of - * the table. Then it gets the list of all tracked files for this table, which may or may not be - * within the table base path, and gets the relative paths of all the tracked files with respect to - * the base of the table. Files outside of the table path will be ignored. Then we take a diff of - * the files and delete directories that were already empty, and all files that are within the table - * that are no longer tracked. - */ -object VacuumCommand extends VacuumCommandImpl with Serializable { - - case class FileNameAndSize(path: String, length: Long, isDir: Boolean) - - /** - * Additional check on retention duration to prevent people from shooting themselves in the foot. - */ - protected def checkRetentionPeriodSafety( - spark: SparkSession, - retentionMs: Option[Long], - configuredRetention: Long): Unit = { - require(retentionMs.forall(_ >= 0), "Retention for Vacuum can't be less than 0.") - val checkEnabled = - spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_RETENTION_CHECK_ENABLED) - val retentionSafe = retentionMs.forall(_ >= configuredRetention) - var configuredRetentionHours = TimeUnit.MILLISECONDS.toHours(configuredRetention) - if (TimeUnit.HOURS.toMillis(configuredRetentionHours) < configuredRetention) { - configuredRetentionHours += 1 - } - require( - !checkEnabled || retentionSafe, - s"""Are you sure you would like to vacuum files with such a low retention period? If you have - |writers that are currently writing to this table, there is a risk that you may corrupt the - |state of your Delta table. - | - |If you are certain that there are no operations being performed on this table, such as - |insert/upsert/delete/optimize, then you may turn off this check by setting: - |spark.databricks.delta.retentionDurationCheck.enabled = false - | - |If you are not sure, please use a value not less than "$configuredRetentionHours hours". - """.stripMargin - ) - } - - /** - * Clears all untracked files and folders within this table. First lists all the files and - * directories in the table, and gets the relative paths with respect to the base of the table. - * Then it gets the list of all tracked files for this table, which may or may not be within the - * table base path, and gets the relative paths of all the tracked files with respect to the base - * of the table. Files outside of the table path will be ignored. Then we take a diff of the files - * and delete directories that were already empty, and all files that are within the table that - * are no longer tracked. - * - * @param dryRun - * If set to true, no files will be deleted. Instead, we will list all files and directories - * that will be cleared. - * @param retentionHours - * An optional parameter to override the default Delta tombstone retention period - * @return - * A Dataset containing the paths of the files/folders to delete in dryRun mode. Otherwise - * returns the base path of the table. - */ - def gc( - spark: SparkSession, - deltaLog: DeltaLog, - dryRun: Boolean = true, - retentionHours: Option[Double] = None, - clock: Clock = new SystemClock): DataFrame = { - recordDeltaOperation(deltaLog, "delta.gc") { - - val path = deltaLog.dataPath - val deltaHadoopConf = deltaLog.newDeltaHadoopConf() - val fs = path.getFileSystem(deltaHadoopConf) - - import org.apache.spark.sql.delta.implicits._ - - val snapshot = deltaLog.update() - - require( - snapshot.version >= 0, - "No state defined for this table. Is this really " + - "a Delta table? Refusing to garbage collect.") - - val retentionMillis = retentionHours.map(h => TimeUnit.HOURS.toMillis(math.round(h))) - checkRetentionPeriodSafety(spark, retentionMillis, deltaLog.tombstoneRetentionMillis) - - val deleteBeforeTimestamp = retentionMillis - .map(millis => clock.getTimeMillis() - millis) - .getOrElse(deltaLog.minFileRetentionTimestamp) - logInfo( - s"Starting garbage collection (dryRun = $dryRun) of untracked files older than " + - s"${new Date(deleteBeforeTimestamp).toString} in $path") - val hadoopConf = spark.sparkContext.broadcast(new SerializableConfiguration(deltaHadoopConf)) - val basePath = fs.makeQualified(path).toString - var isBloomFiltered = false - val parallelDeleteEnabled = - spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_PARALLEL_DELETE_ENABLED) - val parallelDeletePartitions = - spark.sessionState.conf - .getConf(DeltaSQLConf.DELTA_VACUUM_PARALLEL_DELETE_PARALLELISM) - .getOrElse(spark.sessionState.conf.numShufflePartitions) - val relativizeIgnoreError = - spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_RELATIVIZE_IGNORE_ERROR) - val startTimeToIdentifyEligibleFiles = System.currentTimeMillis() - val validFiles = snapshot.stateDS - .mapPartitions { - actions => - val reservoirBase = new Path(basePath) - val fs = reservoirBase.getFileSystem(hadoopConf.value.value) - actions.flatMap { - _.unwrap match { - case tombstone: RemoveFile if tombstone.delTimestamp < deleteBeforeTimestamp => - Nil - case fa: FileAction => - getValidRelativePathsAndSubdirs( - fa, - fs, - reservoirBase, - relativizeIgnoreError, - isBloomFiltered) - case _ => Nil - } - } - } - .toDF("path") - - val partitionColumns = snapshot.metadata.partitionSchema.fieldNames - val parallelism = spark.sessionState.conf.parallelPartitionDiscoveryParallelism - - val allFilesAndDirs = DeltaFileOperations - .recursiveListDirs( - spark, - Seq(basePath), - hadoopConf, - hiddenDirNameFilter = DeltaTableUtils.isHiddenDirectory(partitionColumns, _), - hiddenFileNameFilter = DeltaTableUtils.isHiddenDirectory(partitionColumns, _), - fileListingParallelism = Option(parallelism) - ) - .groupByKey(_.path) - .mapGroups { - (k, v) => - val duplicates = v.toSeq - // of all the duplicates we can return the newest file. - duplicates.maxBy(_.modificationTime) - } - - try { - allFilesAndDirs.cache() - - implicit val fileNameAndSizeEncoder = org.apache.spark.sql.Encoders.product[FileNameAndSize] - - val dirCounts = allFilesAndDirs.where(col("isDir")).count() + 1 // +1 for the base path - - // 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 these - val diff_temp = allFilesAndDirs - .where(col("modificationTime") < deleteBeforeTimestamp || col("isDir")) - .mapPartitions { - fileStatusIterator => - val reservoirBase = new Path(basePath) - val fs = reservoirBase.getFileSystem(hadoopConf.value.value) - fileStatusIterator.flatMap { - fileStatus => - if (fileStatus.isDir) { - Iterator.single( - FileNameAndSize( - relativize(fileStatus.getHadoopPath, fs, reservoirBase, isDir = true), - 0L, - true)) - } else { - val dirs = getAllSubdirs(basePath, fileStatus.path, fs) - val dirsWithSlash = dirs.map { - p => - val relativizedPath = - relativize(new Path(p), fs, reservoirBase, isDir = true) - FileNameAndSize(relativizedPath, 0L, true) - } - dirsWithSlash ++ Iterator( - FileNameAndSize( - relativize(fileStatus.getHadoopPath, fs, reservoirBase, isDir = false), - fileStatus.length, - false)) - } - } - } - .withColumn( - "dir", - when(col("isDir"), col("path")) - .otherwise(expr("substring_index(path, '/',size(split(path, '/')) -1)"))) - .groupBy(col("path"), col("dir")) - .agg(count(new Column("*")).as("count"), sum("length").as("length")) - - val diff = diff_temp - .join(validFiles, diff_temp("dir") === validFiles("path"), "leftanti") - .where(col("count") === 1) - - val sizeOfDataToDeleteRow = diff.agg(sum("length").cast("long")).first - val sizeOfDataToDelete = if (sizeOfDataToDeleteRow.isNullAt(0)) { - 0L - } else { - sizeOfDataToDeleteRow.getLong(0) - } - - val diffFiles = 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)) - } - val timeTakenToIdentifyEligibleFiles = - System.currentTimeMillis() - startTimeToIdentifyEligibleFiles - - if (dryRun) { - val numFiles = diffFiles.count() - val stats = DeltaVacuumStats( - isDryRun = true, - specifiedRetentionMillis = retentionMillis, - defaultRetentionMillis = deltaLog.tombstoneRetentionMillis, - minRetainedTimestamp = deleteBeforeTimestamp, - dirsPresentBeforeDelete = dirCounts, - objectsDeleted = numFiles, - sizeOfDataToDelete = sizeOfDataToDelete, - timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles, - timeTakenForDelete = 0L - ) - - recordDeltaEvent(deltaLog, "delta.gc.stats", data = stats) - logConsole( - s"Found $numFiles files and directories in a total of " + - s"$dirCounts directories that are safe to delete.$stats") - - return diffFiles.map(f => stringToPath(f).toString).toDF("path") - } - logVacuumStart( - spark, - deltaLog, - path, - diffFiles, - sizeOfDataToDelete, - retentionMillis, - deltaLog.tombstoneRetentionMillis) - - val deleteStartTime = System.currentTimeMillis() - val filesDeleted = - try { - delete( - diffFiles, - spark, - basePath, - hadoopConf, - parallelDeleteEnabled, - parallelDeletePartitions) - } catch { - case t: Throwable => - logVacuumEnd(deltaLog, spark, path) - throw t - } - val timeTakenForDelete = System.currentTimeMillis() - deleteStartTime - val stats = DeltaVacuumStats( - isDryRun = false, - specifiedRetentionMillis = retentionMillis, - defaultRetentionMillis = deltaLog.tombstoneRetentionMillis, - minRetainedTimestamp = deleteBeforeTimestamp, - dirsPresentBeforeDelete = dirCounts, - objectsDeleted = filesDeleted, - sizeOfDataToDelete = sizeOfDataToDelete, - timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles, - timeTakenForDelete = timeTakenForDelete - ) - recordDeltaEvent(deltaLog, "delta.gc.stats", data = stats) - logVacuumEnd(deltaLog, spark, path, Some(filesDeleted), Some(dirCounts)) - - spark.createDataset(Seq(basePath)).toDF("path") - } finally { - allFilesAndDirs.unpersist() - } - } - } -} - -trait VacuumCommandImpl extends DeltaCommand { - - protected def logVacuumStart( - spark: SparkSession, - deltaLog: DeltaLog, - path: Path, - diff: Dataset[String], - sizeOfDataToDelete: Long, - specifiedRetentionMillis: Option[Long], - defaultRetentionMillis: Long): Unit = { - logInfo( - s"Deleting untracked files and empty directories in $path. The amount of data to be " + - s"deleted is $sizeOfDataToDelete (in bytes)") - } - - protected def logVacuumEnd( - deltaLog: DeltaLog, - spark: SparkSession, - path: Path, - filesDeleted: Option[Long] = None, - dirCounts: Option[Long] = None): Unit = { - if (filesDeleted.nonEmpty) { - logConsole( - s"Deleted ${filesDeleted.get} files and directories in a total " + - s"of ${dirCounts.get} directories.") - } - } - - /** - * Attempts to relativize the `path` with respect to the `reservoirBase` and converts the path to - * a string. - */ - protected def relativize( - path: Path, - fs: FileSystem, - reservoirBase: Path, - isDir: Boolean): String = { - pathToString(DeltaFileOperations.tryRelativizePath(fs, reservoirBase, path)) - } - - /** - * Wrapper function for DeltaFileOperations.getAllSubDirectories returns all subdirectories that - * `file` has with respect to `base`. - */ - protected def getAllSubdirs(base: String, file: String, fs: FileSystem): Iterator[String] = { - DeltaFileOperations.getAllSubDirectories(base, file)._1 - } - - /** Attempts to delete the list of candidate files. Returns the number of files deleted. */ - protected def delete( - diff: Dataset[String], - spark: SparkSession, - basePath: String, - hadoopConf: Broadcast[SerializableConfiguration], - parallel: Boolean, - parallelPartitions: Int): Long = { - import org.apache.spark.sql.delta.implicits._ - - if (parallel) { - diff - .repartition(parallelPartitions) - .mapPartitions { - files => - val fs = new Path(basePath).getFileSystem(hadoopConf.value.value) - val filesDeletedPerPartition = - files.map(p => stringToPath(p)).count(f => tryDeleteNonRecursive(fs, f)) - Iterator(filesDeletedPerPartition) - } - .collect() - .sum - } else { - val fs = new Path(basePath).getFileSystem(hadoopConf.value.value) - val fileResultSet = diff.toLocalIterator().asScala - fileResultSet.map(p => stringToPath(p)).count(f => tryDeleteNonRecursive(fs, f)) - } - } - - protected def stringToPath(path: String): Path = new Path(new URI(path)) - - protected def pathToString(path: Path): String = path.toUri.toString - - /** Returns the relative path of a file action or None if the file lives outside of the table. */ - protected def getActionRelativePath( - action: FileAction, - fs: FileSystem, - basePath: Path, - relativizeIgnoreError: Boolean): Option[String] = { - val filePath = stringToPath(action.path) - if (filePath.isAbsolute) { - val maybeRelative = - DeltaFileOperations.tryRelativizePath(fs, basePath, filePath, relativizeIgnoreError) - if (maybeRelative.isAbsolute) { - // This file lives outside the directory of the table. - None - } else { - Some(pathToString(maybeRelative)) - } - } else { - Some(pathToString(filePath)) - } - } - - /** - * Returns the relative paths of all files and subdirectories for this action that must be - * retained during GC. - */ - protected def getValidRelativePathsAndSubdirs( - action: FileAction, - fs: FileSystem, - basePath: Path, - relativizeIgnoreError: Boolean, - isBloomFiltered: Boolean): Seq[String] = { - getActionRelativePath(action, fs, basePath, relativizeIgnoreError) - .map(relativePath => Seq(relativePath) ++ getAllSubdirs("/", relativePath, fs)) - .getOrElse(Seq.empty) - } -} - -case class DeltaVacuumStats( - isDryRun: Boolean, - @JsonDeserialize(contentAs = classOf[java.lang.Long]) - specifiedRetentionMillis: Option[Long], - defaultRetentionMillis: Long, - minRetainedTimestamp: Long, - dirsPresentBeforeDelete: Long, - objectsDeleted: Long, - sizeOfDataToDelete: Long, - timeTakenToIdentifyEligibleFiles: Long, - timeTakenForDelete: Long) diff --git a/backends-clickhouse/src/main/delta-22/io/delta/tables/ClickhouseTable.scala b/backends-clickhouse/src/main/delta-23/io/delta/tables/ClickhouseTable.scala similarity index 100% rename from backends-clickhouse/src/main/delta-22/io/delta/tables/ClickhouseTable.scala rename to backends-clickhouse/src/main/delta-23/io/delta/tables/ClickhouseTable.scala diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/DeltaLog.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/DeltaLog.scala similarity index 65% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/DeltaLog.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/DeltaLog.scala index 4cab6454d15a..78fbc3fcdb99 100644 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/DeltaLog.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/DeltaLog.scala @@ -14,15 +14,22 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.delta -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} -import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable} -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expression, Literal} -import org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper -import org.apache.spark.sql.catalyst.util.FailFastMode +// scalastyle:off import.ordering.noEmptyLine +import java.io.File +import java.lang.ref.WeakReference +import java.net.URI +import java.util.concurrent.TimeUnit +import java.util.concurrent.locks.ReentrantLock + +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.util.Try +import scala.util.control.NonFatal + +import com.databricks.spark.util.TagDefinitions._ import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 import org.apache.spark.sql.delta.commands.WriteIntoDelta @@ -30,9 +37,21 @@ import org.apache.spark.sql.delta.commands.cdc.CDCReader import org.apache.spark.sql.delta.files.{TahoeBatchFileIndex, TahoeLogFileIndex} import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.{SchemaMergingUtils, SchemaUtils} -import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.sources._ import org.apache.spark.sql.delta.storage.LogStoreProvider +import com.google.common.cache.{CacheBuilder, RemovalNotification} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable} +import org.apache.spark.sql.catalyst.expressions.{And, Attribute, Cast, Expression, Literal} +import org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper +import org.apache.spark.sql.catalyst.util.FailFastMode import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{BaseRelation, InsertableRelation} @@ -40,56 +59,46 @@ import org.apache.spark.sql.types.{StructField, StructType} import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util._ -// scalastyle:off import.ordering.noEmptyLine -import com.databricks.spark.util.TagDefinitions._ -import com.google.common.cache.{CacheBuilder, RemovalNotification} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} - -import java.io.File -import java.lang.ref.WeakReference -import java.net.URI -import java.util.concurrent.TimeUnit -import java.util.concurrent.locks.ReentrantLock - -import scala.collection.JavaConverters._ -import scala.collection.mutable -import scala.util.Try -import scala.util.control.NonFatal - -// This class is copied from Delta 2.2.0 because it has a private constructor, -// which makes it impossible to extend - /** * Gluten overwrite Delta: * - * This file is copied from Delta 2.2.0 It is modified to overcome the following issues: - * 1. return ClickhouseOptimisticTransaction 2. return DeltaMergeTreeFileFormat + * This file is copied from Delta 2.3.0, it is modified to overcome the following issues: + * 1. return ClickhouseOptimisticTransaction + * 2. return DeltaMergeTreeFileFormat + * 3. create HadoopFsRelation with the bucket options */ - /** - * Used to query the current state of the log as well as modify it by adding new atomic collections - * of actions. + * Used to query the current state of the log as well as modify it by adding + * new atomic collections of actions. * - * Internally, this class implements an optimistic concurrency control algorithm to handle multiple - * readers or writers. Any single read is guaranteed to see a consistent snapshot of the table. + * Internally, this class implements an optimistic concurrency control + * algorithm to handle multiple readers or writers. Any single read + * is guaranteed to see a consistent snapshot of the table. + * + * @param logPath Path of the Delta log JSONs. + * @param dataPath Path of the data files. + * @param options Filesystem options filtered from `allOptions`. + * @param allOptions All options provided by the user, for example via `df.write.option()`. This + * includes but not limited to filesystem and table properties. + * @param clock Clock to be used when starting a new transaction. */ -class DeltaLog private ( +class DeltaLog private( val logPath: Path, val dataPath: Path, val options: Map[String, String], + val allOptions: Map[String, String], val clock: Clock -) extends Checkpoints + ) extends Checkpoints with MetadataCleanup with LogStoreProvider with SnapshotManagement with DeltaFileFormat with ReadChecksum { + import org.apache.spark.sql.delta.util.FileNames._ - import DeltaLog._ - implicit private lazy val _clock = clock + private lazy implicit val _clock = clock protected def spark = SparkSession.active @@ -120,8 +129,7 @@ class DeltaLog private ( /** Delta History Manager containing version and commit history. */ lazy val history = new DeltaHistoryManager( - this, - spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_HISTORY_PAR_SEARCH_THRESHOLD)) + this, spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_HISTORY_PAR_SEARCH_THRESHOLD)) /* --------------- * | Configuration | @@ -129,61 +137,25 @@ class DeltaLog private ( /** * The max lineage length of a Snapshot before Delta forces to build a Snapshot from scratch. - * Delta will build a Snapshot on top of the previous one if it doesn't see a checkpoint. However, - * there is a race condition that when two writers are writing at the same time, a writer may fail - * to pick up checkpoints written by another one, and the lineage will grow and finally cause - * StackOverflowError. Hence we have to force to build a Snapshot from scratch when the lineage - * length is too large to avoid hitting StackOverflowError. + * Delta will build a Snapshot on top of the previous one if it doesn't see a checkpoint. + * However, there is a race condition that when two writers are writing at the same time, + * a writer may fail to pick up checkpoints written by another one, and the lineage will grow + * and finally cause StackOverflowError. Hence we have to force to build a Snapshot from scratch + * when the lineage length is too large to avoid hitting StackOverflowError. */ def maxSnapshotLineageLength: Int = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_MAX_SNAPSHOT_LINEAGE_LENGTH) - /** How long to keep around logically deleted files before physically deleting them. */ - private[delta] def tombstoneRetentionMillis: Long = - DeltaConfigs.getMilliSeconds(DeltaConfigs.TOMBSTONE_RETENTION.fromMetaData(metadata)) - - // TODO: There is a race here where files could get dropped when increasing the - // retention interval... - protected def metadata = Option(unsafeVolatileSnapshot).map(_.metadata).getOrElse(Metadata()) - - /** - * Tombstones before this timestamp will be dropped from the state and the files can be garbage - * collected. - */ - def minFileRetentionTimestamp: Long = { - // TODO (Fred): Get rid of this FrameProfiler record once SC-94033 is addressed - recordFrameProfile("Delta", "DeltaLog.minFileRetentionTimestamp") { - clock.getTimeMillis() - tombstoneRetentionMillis - } - } - - /** - * [[SetTransaction]]s before this timestamp will be considered expired and dropped from the - * state, but no files will be deleted. - */ - def minSetTransactionRetentionTimestamp: Option[Long] = { - DeltaLog.minSetTransactionRetentionInterval(metadata).map(clock.getTimeMillis() - _) - } - - /** - * Checks whether this table only accepts appends. If so it will throw an error in operations that - * can remove data such as DELETE/UPDATE/MERGE. - */ - def assertRemovable(): Unit = { - if (DeltaConfigs.IS_APPEND_ONLY.fromMetaData(metadata)) { - throw DeltaErrors.modifyAppendOnlyTableException(metadata.name) - } - } - /** The unique identifier for this table. */ - def tableId: String = metadata.id + def tableId: String = unsafeVolatileMetadata.id // safe because table id never changes /** - * Combines the tableId with the path of the table to ensure uniqueness. Normally `tableId` should - * be globally unique, but nothing stops users from copying a Delta table directly to a separate - * location, where the transaction log is copied directly, causing the tableIds to match. When - * users mutate the copied table, and then try to perform some checks joining the two tables, - * optimizations that depend on `tableId` alone may not be correct. Hence we use a composite id. + * Combines the tableId with the path of the table to ensure uniqueness. Normally `tableId` + * should be globally unique, but nothing stops users from copying a Delta table directly to + * a separate location, where the transaction log is copied directly, causing the tableIds to + * match. When users mutate the copied table, and then try to perform some checks joining the + * two tables, optimizations that depend on `tableId` alone may not be correct. Hence we use a + * composite id. */ private[delta] def compositeId: (String, Path) = tableId -> dataPath @@ -224,19 +196,32 @@ class DeltaLog private ( "ignoreCorruptFiles" -> "false", "ignoreMissingFiles" -> "false" ) - val fsRelation = - HadoopFsRelation(index, index.partitionSchema, schema, None, index.format, allOptions)(spark) + // --- modified start + // Don't need to add the bucketOption here, it handles the delta log meta json file + // --- modified end + val fsRelation = HadoopFsRelation( + index, index.partitionSchema, schema, None, index.format, allOptions)(spark) LogicalRelation(fsRelation) } + /** + * Load the data using the FileIndex. This allows us to skip many checks that add overhead, e.g. + * file existence checks, partitioning schema inference. + */ + def loadIndex( + index: DeltaLogFileIndex, + schema: StructType = Action.logSchema): DataFrame = { + Dataset.ofRows(spark, indexToRelation(index, schema)) + } + /* ------------------ * | Delta Management | * ------------------ */ /** - * Returns a new [[OptimisticTransaction]] that can be used to read the current state of the log - * and then commit updates. The reads and updates will be checked for logical conflicts with any - * concurrent writes to the log. + * Returns a new [[OptimisticTransaction]] that can be used to read the current state of the + * log and then commit updates. The reads and updates will be checked for logical conflicts + * with any concurrent writes to the log. * * Note that all reads in a transaction must go through the returned transaction object, and not * directly to the [[DeltaLog]] otherwise they will not be checked for conflicts. @@ -244,17 +229,18 @@ class DeltaLog private ( def startTransaction(): OptimisticTransaction = startTransaction(None) def startTransaction(snapshotOpt: Option[Snapshot]): OptimisticTransaction = { + // --- modified start new ClickhouseOptimisticTransaction(this, snapshotOpt) + // --- modified end } /** - * Execute a piece of code within a new [[OptimisticTransaction]]. Reads/write sets will be - * recorded for this table, and all other tables will be read at a snapshot that is pinned on the - * first access. + * Execute a piece of code within a new [[OptimisticTransaction]]. Reads/write sets will + * be recorded for this table, and all other tables will be read + * at a snapshot that is pinned on the first access. * - * @note - * This uses thread-local variable to make the active transaction visible. So do not use - * multi-threaded code in the provided thunk. + * @note This uses thread-local variable to make the active transaction visible. So do not use + * multi-threaded code in the provided thunk. */ def withNewTransaction[T](thunk: OptimisticTransaction => T): T = { try { @@ -266,16 +252,16 @@ class DeltaLog private ( } } + /** * Upgrade the table's protocol version, by default to the maximum recognized reader and writer * versions in this DBR release. */ - def upgradeProtocol(snapshot: Snapshot, newVersion: Protocol): Unit = { + def upgradeProtocol( + snapshot: Snapshot, + newVersion: Protocol): Unit = { val currentVersion = snapshot.protocol - if ( - newVersion.minReaderVersion == currentVersion.minReaderVersion && - newVersion.minWriterVersion == currentVersion.minWriterVersion - ) { + if (newVersion == currentVersion) { logConsole(s"Table $dataPath is already at protocol version $newVersion.") return } @@ -292,7 +278,7 @@ class DeltaLog private ( } // Test-only!! - private[delta] def upgradeProtocol(newVersion: Protocol = Protocol()): Unit = { + private[delta] def upgradeProtocol(newVersion: Protocol): Unit = { upgradeProtocol(unsafeVolatileSnapshot, newVersion) } @@ -304,41 +290,39 @@ class DeltaLog private ( startVersion: Long, failOnDataLoss: Boolean = false): Iterator[(Long, Seq[Action])] = { val hadoopConf = newDeltaHadoopConf() - val deltas = store.listFrom(deltaFile(logPath, startVersion), hadoopConf).filter(isDeltaFile) + val deltas = store.listFrom(listingPrefix(logPath, startVersion), hadoopConf) + .filter(isDeltaFile) // Subtract 1 to ensure that we have the same check for the inclusive startVersion var lastSeenVersion = startVersion - 1 - deltas.map { - status => - val p = status.getPath - val version = deltaVersion(p) - if (failOnDataLoss && version > lastSeenVersion + 1) { - throw DeltaErrors.failOnDataLossException(lastSeenVersion + 1, version) - } - lastSeenVersion = version - (version, store.read(status, hadoopConf).map(Action.fromJson)) + deltas.map { status => + val p = status.getPath + val version = deltaVersion(p) + if (failOnDataLoss && version > lastSeenVersion + 1) { + throw DeltaErrors.failOnDataLossException(lastSeenVersion + 1, version) + } + lastSeenVersion = version + (version, store.read(status, hadoopConf).map(Action.fromJson)) } } /** - * Get access to all actions starting from "startVersion" (inclusive) via [[FileStatus]]. If - * `startVersion` doesn't exist, return an empty Iterator. + * Get access to all actions starting from "startVersion" (inclusive) via [[FileStatus]]. + * If `startVersion` doesn't exist, return an empty Iterator. */ def getChangeLogFiles( startVersion: Long, failOnDataLoss: Boolean = false): Iterator[(Long, FileStatus)] = { - val deltas = store - .listFrom(deltaFile(logPath, startVersion), newDeltaHadoopConf()) + val deltas = store.listFrom(listingPrefix(logPath, startVersion), newDeltaHadoopConf()) .filter(isDeltaFile) // Subtract 1 to ensure that we have the same check for the inclusive startVersion var lastSeenVersion = startVersion - 1 - deltas.map { - status => - val version = deltaVersion(status) - if (failOnDataLoss && version > lastSeenVersion + 1) { - throw DeltaErrors.failOnDataLossException(lastSeenVersion + 1, version) - } - lastSeenVersion = version - (version, status) + deltas.map { status => + val version = deltaVersion(status) + if (failOnDataLoss && version > lastSeenVersion + 1) { + throw DeltaErrors.failOnDataLossException(lastSeenVersion + 1, version) + } + lastSeenVersion = version + (version, status) } } @@ -346,40 +330,108 @@ class DeltaLog private ( | Protocol validation | * --------------------- */ + /** + * Asserts the highest protocol supported by this client is not less than what required by the + * table for performing read or write operations. This ensures the client to support a + * greater-or-equal protocol versions and recognizes/supports all features enabled by the table. + * + * The operation type to be checked is passed as a string in `readOrWrite`. Valid values are + * `read` and `write`. + */ + private def protocolCheck(tableProtocol: Protocol, readOrWrite: String): Unit = { + val clientSupportedProtocol = Action.supportedProtocolVersion() + // Depending on the operation, pull related protocol versions out of Protocol objects. + // `getEnabledFeatures` is a pointer to pull reader/writer features out of a Protocol. + val (clientSupportedVersion, tableRequiredVersion, getEnabledFeatures) = readOrWrite match { + case "read" => ( + clientSupportedProtocol.minReaderVersion, + tableProtocol.minReaderVersion, + (f: Protocol) => f.readerFeatureNames) + case "write" => ( + clientSupportedProtocol.minWriterVersion, + tableProtocol.minWriterVersion, + (f: Protocol) => f.writerFeatureNames) + case _ => + throw new IllegalArgumentException("Table operation must be either `read` or `write`.") + } + + // Check is complete when both the protocol version and all referenced features are supported. + val clientSupportedFeatureNames = getEnabledFeatures(clientSupportedProtocol) + val tableEnabledFeatureNames = getEnabledFeatures(tableProtocol) + if (tableEnabledFeatureNames.subsetOf(clientSupportedFeatureNames) && + clientSupportedVersion >= tableRequiredVersion) { + return + } + + // Otherwise, either the protocol version, or few features referenced by the table, is + // unsupported. + val clientUnsupportedFeatureNames = + tableEnabledFeatureNames.diff(clientSupportedFeatureNames) + // Prepare event log constants and the appropriate error message handler. + val (opType, versionKey, unsupportedFeaturesException) = readOrWrite match { + case "read" => ( + "delta.protocol.failure.read", + "minReaderVersion", + DeltaErrors.unsupportedReaderTableFeaturesInTableException _) + case "write" => ( + "delta.protocol.failure.write", + "minWriterVersion", + DeltaErrors.unsupportedWriterTableFeaturesInTableException _) + } + recordDeltaEvent( + this, + opType, + data = Map( + "clientVersion" -> clientSupportedVersion, + versionKey -> tableRequiredVersion, + "clientFeatures" -> clientSupportedFeatureNames.mkString(","), + "clientUnsupportedFeatures" -> clientUnsupportedFeatureNames.mkString(","))) + if (clientSupportedVersion < tableRequiredVersion) { + throw new InvalidProtocolVersionException(tableRequiredVersion, clientSupportedVersion) + } else { + throw unsupportedFeaturesException(clientUnsupportedFeatureNames) + } + } + + /** + * Asserts that the table's protocol enabled all features that are active in the metadata. + * + * A mismatch shouldn't happen when the table has gone through a proper write process because we + * require all active features during writes. However, other clients may void this guarantee. + */ + def assertTableFeaturesMatchMetadata( + targetProtocol: Protocol, + targetMetadata: Metadata): Unit = { + if (!targetProtocol.supportsReaderFeatures && !targetProtocol.supportsWriterFeatures) return + + val protocolEnabledFeatures = targetProtocol.writerFeatureNames + .flatMap(TableFeature.featureNameToFeature) + val activeFeatures: Set[TableFeature] = + TableFeature.allSupportedFeaturesMap.values.collect { + case f: TableFeature with FeatureAutomaticallyEnabledByMetadata + if f.metadataRequiresFeatureToBeEnabled(targetMetadata, spark) => + f + }.toSet + val activeButNotEnabled = activeFeatures.diff(protocolEnabledFeatures) + if (activeButNotEnabled.nonEmpty) { + throw DeltaErrors.tableFeatureMismatchException(activeButNotEnabled.map(_.name)) + } + } + /** * Asserts that the client is up to date with the protocol and allowed to read the table that is * using the given `protocol`. */ def protocolRead(protocol: Protocol): Unit = { - val supportedReaderVersion = - Action.supportedProtocolVersion(Some(spark.sessionState.conf)).minReaderVersion - if (supportedReaderVersion < protocol.minReaderVersion) { - recordDeltaEvent( - this, - "delta.protocol.failure.read", - data = Map( - "clientVersion" -> supportedReaderVersion, - "minReaderVersion" -> protocol.minReaderVersion)) - throw new InvalidProtocolVersionException - } + protocolCheck(protocol, "read") } /** - * Asserts that the client is up to date with the protocol and allowed to write to the table that - * is using the given `protocol`. + * Asserts that the client is up to date with the protocol and allowed to write to the table + * that is using the given `protocol`. */ - def protocolWrite(protocol: Protocol, logUpgradeMessage: Boolean = true): Unit = { - val supportedWriterVersion = - Action.supportedProtocolVersion(Some(spark.sessionState.conf)).minWriterVersion - if (supportedWriterVersion < protocol.minWriterVersion) { - recordDeltaEvent( - this, - "delta.protocol.failure.write", - data = Map( - "clientVersion" -> supportedWriterVersion, - "minWriterVersion" -> protocol.minWriterVersion)) - throw new InvalidProtocolVersionException - } + def protocolWrite(protocol: Protocol): Unit = { + protocolCheck(protocol, "write") } /* ---------------------------------------- * @@ -387,9 +439,10 @@ class DeltaLog private ( * ---------------------------------------- */ /** - * Whether a Delta table exists at this directory. It is okay to use the cached volatile snapshot - * here, since the worst case is that the table has recently started existing which hasn't been - * picked up here. If so, any subsequent command that updates the table will see the right value. + * Whether a Delta table exists at this directory. + * It is okay to use the cached volatile snapshot here, since the worst case is that the table + * has recently started existing which hasn't been picked up here. If so, any subsequent command + * that updates the table will see the right value. */ def tableExists: Boolean = unsafeVolatileSnapshot.version >= 0 @@ -420,38 +473,46 @@ class DeltaLog private ( /** * Returns a [[org.apache.spark.sql.DataFrame]] containing the new files within the specified * version range. + * */ def createDataFrame( snapshot: Snapshot, addFiles: Seq[AddFile], isStreaming: Boolean = false, - actionTypeOpt: Option[String] = None): DataFrame = { + actionTypeOpt: Option[String] = None + ): DataFrame = { val actionType = actionTypeOpt.getOrElse(if (isStreaming) "streaming" else "batch") val fileIndex = new TahoeBatchFileIndex(spark, actionType, addFiles, this, dataPath, snapshot) val hadoopOptions = snapshot.metadata.format.options ++ options + val partitionSchema = snapshot.metadata.partitionSchema + val metadata = snapshot.metadata + val relation = HadoopFsRelation( fileIndex, - partitionSchema = - DeltaColumnMapping.dropColumnMappingMetadata(snapshot.metadata.partitionSchema), + partitionSchema = DeltaColumnMapping.dropColumnMappingMetadata(partitionSchema), // We pass all table columns as `dataSchema` so that Spark will preserve the partition column // locations. Otherwise, for any partition columns not in `dataSchema`, Spark would just // append them to the end of `dataSchema`. dataSchema = DeltaColumnMapping.dropColumnMappingMetadata( - ColumnWithDefaultExprUtils.removeDefaultExpressions(snapshot.metadata.schema)), + ColumnWithDefaultExprUtils.removeDefaultExpressions(metadata.schema)), + // --- modified start + // TODO: Don't add the bucketOption here, it will cause the OOM when the merge into update + // key is the bucket column, fix later + // --- modified end bucketSpec = None, - snapshot.deltaLog.fileFormat(snapshot.metadata), - hadoopOptions - )(spark) + fileFormat(metadata), + hadoopOptions)(spark) Dataset.ofRows(spark, LogicalRelation(relation, isStreaming = isStreaming)) } /** - * Returns a [[BaseRelation]] that contains all of the data present in the table. This relation - * will be continually updated as files are added or removed from the table. However, new - * [[BaseRelation]] must be requested in order to see changes to the schema. + * Returns a [[BaseRelation]] that contains all of the data present + * in the table. This relation will be continually updated + * as files are added or removed from the table. However, new [[BaseRelation]] + * must be requested in order to see changes to the schema. */ def createRelation( partitionFilters: Seq[Expression] = Nil, @@ -473,21 +534,23 @@ class DeltaLog private ( if (!cdcOptions.isEmpty) { recordDeltaEvent(this, "delta.cdf.read", data = cdcOptions.asCaseSensitiveMap()) return CDCReader.getCDCRelation( - spark, - this, - snapshotToUse, - partitionFilters, - spark.sessionState.conf, - cdcOptions) + spark, snapshotToUse, isTimeTravelQuery, spark.sessionState.conf, cdcOptions) } - val fileIndex = - TahoeLogFileIndex(spark, this, dataPath, snapshotToUse, partitionFilters, isTimeTravelQuery) - var bucketSpec: Option[BucketSpec] = ClickHouseTableV2.getTable(this).bucketOption - new DeltaHadoopFsRelation( + val fileIndex = TahoeLogFileIndex( + spark, this, dataPath, snapshotToUse, partitionFilters, isTimeTravelQuery) + // --- modified start + var bucketSpec: Option[BucketSpec] = + if (ClickHouseConfig.isMergeTreeFormatEngine(snapshotToUse.metadata.configuration)) { + ClickHouseTableV2.getTable(this).bucketOption + } else { + None + } + + new DeltaLog.DeltaHadoopFsRelation( fileIndex, - partitionSchema = - DeltaColumnMapping.dropColumnMappingMetadata(snapshotToUse.metadata.partitionSchema), + partitionSchema = DeltaColumnMapping.dropColumnMappingMetadata( + snapshotToUse.metadata.partitionSchema), // We pass all table columns as `dataSchema` so that Spark will preserve the partition column // locations. Otherwise, for any partition columns not in `dataSchema`, Spark would just // append them to the end of `dataSchema` @@ -504,18 +567,21 @@ class DeltaLog private ( spark, this ) + // --- modified end } /** - * Verify the required Spark conf for delta Throw - * `DeltaErrors.configureSparkSessionWithExtensionAndCatalog` exception if - * `spark.sql.catalog.spark_catalog` config is missing. We do not check for `spark.sql.extensions` - * because DeltaSparkSessionExtension can alternatively be activated using the `.withExtension()` - * API. This check can be disabled by setting DELTA_CHECK_REQUIRED_SPARK_CONF to false. + * Verify the required Spark conf for delta + * Throw `DeltaErrors.configureSparkSessionWithExtensionAndCatalog` exception if + * `spark.sql.catalog.spark_catalog` config is missing. We do not check for + * `spark.sql.extensions` because DeltaSparkSessionExtension can alternatively + * be activated using the `.withExtension()` API. This check can be disabled + * by setting DELTA_CHECK_REQUIRED_SPARK_CONF to false. */ protected def checkRequiredConfigurations(): Unit = { if (spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_REQUIRED_SPARK_CONFS_CHECK)) { - if (spark.conf.getOption(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key).isEmpty) { + if (spark.conf.getOption( + SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key).isEmpty) { throw DeltaErrors.configureSparkSessionWithExtensionAndCatalog(None) } } @@ -524,9 +590,9 @@ class DeltaLog private ( /** * Returns a proper path canonicalization function for the current Delta log. * - * If `runsOnExecutors` is true, the returned method will use a broadcast Hadoop Configuration so - * that the method is suitable for execution on executors. Otherwise, the returned method will use - * a local Hadoop Configuration and the method can only be executed on the driver. + * If `runsOnExecutors` is true, the returned method will use a broadcast Hadoop Configuration + * so that the method is suitable for execution on executors. Otherwise, the returned method + * will use a local Hadoop Configuration and the method can only be executed on the driver. */ private[delta] def getCanonicalPathFunction(runsOnExecutors: Boolean): String => String = { val hadoopConf = newDeltaHadoopConf() @@ -535,7 +601,9 @@ class DeltaLog private ( val broadcastHadoopConf = spark.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) () => broadcastHadoopConf.value.value - } else { () => hadoopConf } + } else { + () => hadoopConf + } new DeltaLog.CanonicalPathFunction(getHadoopConf) } @@ -544,24 +612,33 @@ class DeltaLog private ( * Returns a proper path canonicalization UDF for the current Delta log. * * If `runsOnExecutors` is true, the returned UDF will use a broadcast Hadoop Configuration. - * Otherwise, the returned UDF will use a local Hadoop Configuration and the UDF can only be - * executed on the driver. + * Otherwise, the returned UDF will use a local Hadoop Configuration and the UDF can + * only be executed on the driver. */ private[delta] def getCanonicalPathUdf(runsOnExecutors: Boolean = true): UserDefinedFunction = { DeltaUDF.stringFromString(getCanonicalPathFunction(runsOnExecutors)) } - override def fileFormat(metadata: Metadata = metadata): FileFormat = - ClickHouseTableV2.getTable(this).getFileFormat(metadata) + override def fileFormat(metadata: Metadata): FileFormat = { + // --- modified start + if (ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { + ClickHouseTableV2.getTable(this).getFileFormat(metadata) + } else { + super.fileFormat(metadata) + } + // --- modified end + } } object DeltaLog extends DeltaLogging { + + // --- modified start @SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass")) private class DeltaHadoopFsRelation( location: FileIndex, partitionSchema: StructType, - // The top-level columns in `dataSchema` should match the actual physical file schema, otherwise - // the ORC data source may not work with the by-ordinal mode. + // The top-level columns in `dataSchema` should match the actual physical file schema, + // otherwise the ORC data source may not work with the by-ordinal mode. dataSchema: StructType, bucketSpec: Option[BucketSpec], fileFormat: FileFormat, @@ -573,7 +650,7 @@ object DeltaLog extends DeltaLogging { bucketSpec, fileFormat, options)(sparkSession) - with InsertableRelation { + with InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit = { val mode = if (overwrite) SaveMode.Overwrite else SaveMode.Append WriteIntoDelta( @@ -586,6 +663,7 @@ object DeltaLog extends DeltaLogging { ).run(sparkSession) } } + // --- modified end /** * The key type of `DeltaLog` cache. It's a pair of the canonicalized table path and the file @@ -602,30 +680,27 @@ object DeltaLog extends DeltaLogging { private[delta] def logPathFor(dataPath: File): Path = logPathFor(dataPath.getAbsolutePath) /** - * We create only a single [[DeltaLog]] for any given `DeltaLogCacheKey` to avoid wasted work in - * reconstructing the log. + * We create only a single [[DeltaLog]] for any given `DeltaLogCacheKey` to avoid wasted work + * in reconstructing the log. */ private val deltaLogCache = { - val builder = CacheBuilder - .newBuilder() + val builder = CacheBuilder.newBuilder() .expireAfterAccess(60, TimeUnit.MINUTES) - .removalListener( - (removalNotification: RemovalNotification[DeltaLogCacheKey, DeltaLog]) => { + .removalListener((removalNotification: RemovalNotification[DeltaLogCacheKey, DeltaLog]) => { val log = removalNotification.getValue // TODO: We should use ref-counting to uncache snapshots instead of a manual timed op - try log.unsafeVolatileSnapshot.uncache() - catch { + try log.unsafeVolatileSnapshot.uncache() catch { case _: java.lang.NullPointerException => // Various layers will throw null pointer if the RDD is already gone. } - }) - sys.props - .get("delta.log.cacheSize") + }) + sys.props.get("delta.log.cacheSize") .flatMap(v => Try(v.toLong).toOption) .foreach(builder.maximumSize) builder.build[DeltaLogCacheKey, DeltaLog]() } + // Don't tolerate malformed JSON when parsing Delta log actions (default is PERMISSIVE) val jsonCommitParseOption = Map("mode" -> FailFastMode.name) @@ -710,28 +785,38 @@ object DeltaLog extends DeltaLogging { private def apply(spark: SparkSession, rawPath: Path, clock: Clock = new SystemClock): DeltaLog = apply(spark, rawPath, Map.empty, clock) + /** Helper for getting a log, as well as the latest snapshot, of the table */ def forTableWithSnapshot(spark: SparkSession, dataPath: String): (DeltaLog, Snapshot) = - withFreshSnapshot(forTable(spark, dataPath, _)) + withFreshSnapshot { forTable(spark, dataPath, _) } /** Helper for getting a log, as well as the latest snapshot, of the table */ def forTableWithSnapshot(spark: SparkSession, dataPath: Path): (DeltaLog, Snapshot) = - withFreshSnapshot(forTable(spark, dataPath, _)) + withFreshSnapshot { forTable(spark, dataPath, _) } /** Helper for getting a log, as well as the latest snapshot, of the table */ - def forTableWithSnapshot(spark: SparkSession, tableName: TableIdentifier): (DeltaLog, Snapshot) = - withFreshSnapshot(forTable(spark, tableName, _)) + def forTableWithSnapshot( + spark: SparkSession, + tableName: TableIdentifier): (DeltaLog, Snapshot) = + withFreshSnapshot { forTable(spark, tableName, _) } /** Helper for getting a log, as well as the latest snapshot, of the table */ def forTableWithSnapshot( spark: SparkSession, tableName: DeltaTableIdentifier): (DeltaLog, Snapshot) = - withFreshSnapshot(forTable(spark, tableName, _)) + withFreshSnapshot { forTable(spark, tableName, _) } + + /** Helper for getting a log, as well as the latest snapshot, of the table */ + def forTableWithSnapshot( + spark: SparkSession, + dataPath: Path, + options: Map[String, String]): (DeltaLog, Snapshot) = + withFreshSnapshot { apply(spark, logPathFor(dataPath), options, _) } /** - * Helper function to be used with the forTableWithSnapshot calls. Thunk is a partially applied - * DeltaLog.forTable call, which we can then wrap around with a snapshot update. We use the system - * clock to avoid back-to-back updates. + * Helper function to be used with the forTableWithSnapshot calls. Thunk is a + * partially applied DeltaLog.forTable call, which we can then wrap around with a + * snapshot update. We use the system clock to avoid back-to-back updates. */ private[delta] def withFreshSnapshot(thunk: Clock => DeltaLog): (DeltaLog, Snapshot) = { val clock = new SystemClock @@ -748,14 +833,12 @@ object DeltaLog extends DeltaLogging { clock: Clock ): DeltaLog = { val fileSystemOptions: Map[String, String] = - if ( - spark.sessionState.conf.getConf( - DeltaSQLConf.LOAD_FILE_SYSTEM_CONFIGS_FROM_DATAFRAME_OPTIONS) - ) { + if (spark.sessionState.conf.getConf( + DeltaSQLConf.LOAD_FILE_SYSTEM_CONFIGS_FROM_DATAFRAME_OPTIONS)) { // We pick up only file system options so that we don't pass any parquet or json options to // the code that reads Delta transaction logs. - options.filterKeys { - k => DeltaTableUtils.validDeltaTableHadoopPrefixes.exists(k.startsWith) + options.filterKeys { k => + DeltaTableUtils.validDeltaTableHadoopPrefixes.exists(k.startsWith) }.toMap } else { Map.empty @@ -769,14 +852,15 @@ object DeltaLog extends DeltaLogging { null, "delta.log.create", Map(TAG_TAHOE_PATH -> path.getParent.toString)) { - AnalysisHelper.allowInvokingTransformsInAnalyzer { - new DeltaLog( - logPath = path, - dataPath = path.getParent, - options = fileSystemOptions, - clock = clock - ) - } + AnalysisHelper.allowInvokingTransformsInAnalyzer { + new DeltaLog( + logPath = path, + dataPath = path.getParent, + options = fileSystemOptions, + allOptions = options, + clock = clock + ) + } } def getDeltaLogFromCache(): DeltaLog = { // The following cases will still create a new ActionLog even if there is a cached @@ -785,7 +869,10 @@ object DeltaLog extends DeltaLogging { // - Different `authority` (e.g., different user tokens in the path) // - Different mount point. try { - deltaLogCache.get(path -> fileSystemOptions, () => createDeltaLog()) + deltaLogCache.get(path -> fileSystemOptions, () => { + createDeltaLog() + } + ) } catch { case e: com.google.common.util.concurrent.UncheckedExecutionException => throw e.getCause @@ -814,10 +901,8 @@ object DeltaLog extends DeltaLogging { // scalastyle:on deltahadoopconfiguration val path = fs.makeQualified(rawPath) - if ( - spark.sessionState.conf.getConf( - DeltaSQLConf.LOAD_FILE_SYSTEM_CONFIGS_FROM_DATAFRAME_OPTIONS) - ) { + if (spark.sessionState.conf.getConf( + DeltaSQLConf.LOAD_FILE_SYSTEM_CONFIGS_FROM_DATAFRAME_OPTIONS)) { // We rely on the fact that accessing the key set doesn't modify the entry access time. See // `CacheBuilder.expireAfterAccess`. val keysToBeRemoved = mutable.ArrayBuffer[DeltaLogCacheKey]() @@ -848,38 +933,42 @@ object DeltaLog extends DeltaLogging { /** * Filters the given [[Dataset]] by the given `partitionFilters`, returning those that match. - * @param files - * The active files in the DeltaLog state, which contains the partition value information - * @param partitionFilters - * Filters on the partition columns - * @param partitionColumnPrefixes - * The path to the `partitionValues` column, if it's nested + * @param files The active files in the DeltaLog state, which contains the partition value + * information + * @param partitionFilters Filters on the partition columns + * @param partitionColumnPrefixes The path to the `partitionValues` column, if it's nested + * @param shouldRewritePartitionFilters Whether to rewrite `partitionFilters` to be over the + * [[AddFile]] schema */ def filterFileList( partitionSchema: StructType, files: DataFrame, partitionFilters: Seq[Expression], - partitionColumnPrefixes: Seq[String] = Nil): DataFrame = { - val rewrittenFilters = rewritePartitionFilters( - partitionSchema, - files.sparkSession.sessionState.conf.resolver, - partitionFilters, - partitionColumnPrefixes) + partitionColumnPrefixes: Seq[String] = Nil, + shouldRewritePartitionFilters: Boolean = true): DataFrame = { + + val rewrittenFilters = if (shouldRewritePartitionFilters) { + rewritePartitionFilters( + partitionSchema, + files.sparkSession.sessionState.conf.resolver, + partitionFilters, + partitionColumnPrefixes) + } else { + partitionFilters + } val expr = rewrittenFilters.reduceLeftOption(And).getOrElse(Literal.TrueLiteral) val columnFilter = new Column(expr) files.filter(columnFilter) } /** - * Rewrite the given `partitionFilters` to be used for filtering partition values. We need to - * explicitly resolve the partitioning columns here because the partition columns are stored as - * keys of a Map type instead of attributes in the AddFile schema (below) and thus cannot be - * resolved automatically. + * Rewrite the given `partitionFilters` to be used for filtering partition values. + * We need to explicitly resolve the partitioning columns here because the partition columns + * are stored as keys of a Map type instead of attributes in the AddFile schema (below) and thus + * cannot be resolved automatically. * - * @param partitionFilters - * Filters on the partition columns - * @param partitionColumnPrefixes - * The path to the `partitionValues` column, if it's nested + * @param partitionFilters Filters on the partition columns + * @param partitionColumnPrefixes The path to the `partitionValues` column, if it's nested */ def rewritePartitionFilters( partitionSchema: StructType, @@ -891,7 +980,7 @@ object DeltaLog extends DeltaLogging { // If we have a special column name, e.g. `a.a`, then an UnresolvedAttribute returns // the column name as '`a.a`' instead of 'a.a', therefore we need to strip the backticks. val unquoted = a.name.stripPrefix("`").stripSuffix("`") - val partitionCol = partitionSchema.find(field => resolver(field.name, unquoted)) + val partitionCol = partitionSchema.find { field => resolver(field.name, unquoted) } partitionCol match { case Some(f: StructField) => val name = DeltaColumnMapping.getPhysicalName(f) @@ -907,16 +996,32 @@ object DeltaLog extends DeltaLogging { }) } + + /** + * Checks whether this table only accepts appends. If so it will throw an error in operations that + * can remove data such as DELETE/UPDATE/MERGE. + */ + def assertRemovable(snapshot: Snapshot): Unit = { + val metadata = snapshot.metadata + if (DeltaConfigs.IS_APPEND_ONLY.fromMetaData(metadata)) { + throw DeltaErrors.modifyAppendOnlyTableException(metadata.name) + } + } + + /** How long to keep around SetTransaction actions before physically deleting them. */ def minSetTransactionRetentionInterval(metadata: Metadata): Option[Long] = { DeltaConfigs.TRANSACTION_ID_RETENTION_DURATION .fromMetaData(metadata) .map(DeltaConfigs.getMilliSeconds) } + /** How long to keep around logically deleted files before physically deleting them. */ + def tombstoneRetentionMillis(metadata: Metadata): Long = { + DeltaConfigs.getMilliSeconds(DeltaConfigs.TOMBSTONE_RETENTION.fromMetaData(metadata)) + } /** Get a function that canonicalizes a given `path`. */ private[delta] class CanonicalPathFunction(getHadoopConf: () => Configuration) - extends Function[String, String] - with Serializable { + extends Function[String, String] with Serializable { // Mark it `@transient lazy val` so that de-serialization happens only once on every executor. @transient private lazy val fs = { diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/Snapshot.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/Snapshot.scala similarity index 59% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/Snapshot.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/Snapshot.scala index 1c62d133174d..b2b5ba42bb30 100644 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/Snapshot.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/Snapshot.scala @@ -14,33 +14,38 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.delta -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.Expression +// scalastyle:off import.ordering.noEmptyLine +import scala.collection.mutable + import org.apache.spark.sql.delta.actions._ import org.apache.spark.sql.delta.actions.Action.logSchema import org.apache.spark.sql.delta.metering.DeltaLogging import org.apache.spark.sql.delta.schema.SchemaUtils import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.stats.{DataSkippingReader, DeltaScan, FileSizeHistogram, StatisticsCollection} +import org.apache.spark.sql.delta.stats.DataSkippingReader +import org.apache.spark.sql.delta.stats.DeltaScan +import org.apache.spark.sql.delta.stats.FileSizeHistogram +import org.apache.spark.sql.delta.stats.StatisticsCollection import org.apache.spark.sql.delta.util.StateCache +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -import org.apache.hadoop.fs.{FileStatus, Path} - -// scalastyle:off import.ordering.noEmptyLine -import scala.collection.mutable - /** * Gluten overwrite Delta: * - * This file is copied from Delta 2.2.0. It is modified to overcome the following issues: - * 1. filesForScan() should return DeltaScan of AddMergeTreeParts instead of AddFile + * This file is copied from Delta 2.3.0. It is modified to overcome the following issues: + * 1. filesForScan() will cache the DeltaScan by the FilterExprsAsKey + * 2. filesForScan() should return DeltaScan of AddMergeTreeParts instead of AddFile */ - /** * A description of a Delta [[Snapshot]], including basic information such its [[DeltaLog]] * metadata, protocol, and version. @@ -55,28 +60,27 @@ trait SnapshotDescriptor { } /** - * An immutable snapshot of the state of the log at some delta version. Internally this class - * manages the replay of actions stored in checkpoint or delta files. + * An immutable snapshot of the state of the log at some delta version. Internally + * this class manages the replay of actions stored in checkpoint or delta files. * - * After resolving any new actions, it caches the result and collects the following basic - * information to the driver: - * - Protocol Version - * - Metadata - * - Transaction state + * After resolving any new actions, it caches the result and collects the + * following basic information to the driver: + * - Protocol Version + * - Metadata + * - Transaction state + * + * @param timestamp The timestamp of the latest commit in milliseconds. Can also be set to -1 if the + * timestamp of the commit is unknown or the table has not been initialized, i.e. + * `version = -1`. * - * @param timestamp - * The timestamp of the latest commit in milliseconds. Can also be set to -1 if the timestamp of - * the commit is unknown or the table has not been initialized, i.e. `version = -1`. */ class Snapshot( val path: Path, override val version: Long, val logSegment: LogSegment, - val minFileRetentionTimestamp: Long, override val deltaLog: DeltaLog, val timestamp: Long, val checksumOpt: Option[VersionChecksum], - val minSetTransactionRetentionTimestamp: Option[Long] = None, checkpointMetadataOpt: Option[CheckpointMetaData] = None) extends SnapshotDescriptor with StateCache @@ -84,25 +88,25 @@ class Snapshot( with DataSkippingReader with DeltaLogging { - import org.apache.spark.sql.delta.implicits._ - - // For implicits which re-use Encoder: import Snapshot._ + // For implicits which re-use Encoder: + import org.apache.spark.sql.delta.implicits._ protected def spark = SparkSession.active + /** Snapshot to scan by the DeltaScanGenerator for metadata query optimizations */ override val snapshotToScan: Snapshot = this protected def getNumPartitions: Int = { - spark.sessionState.conf - .getConf(DeltaSQLConf.DELTA_SNAPSHOT_PARTITIONS) + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_SNAPSHOT_PARTITIONS) .getOrElse(Snapshot.defaultNumSnapshotPartitions) } /** Performs validations during initialization */ protected def init(): Unit = { deltaLog.protocolRead(protocol) + deltaLog.assertTableFeaturesMatchMetadata(protocol, metadata) SchemaUtils.recordUndefinedTypes(deltaLog, metadata.schema) } @@ -127,49 +131,61 @@ class Snapshot( val ADD_PATH_CANONICAL_COL_NAME = "add_path_canonical" val REMOVE_PATH_CANONICAL_COL_NAME = "remove_path_canonical" loadActions - .withColumn( - ADD_PATH_CANONICAL_COL_NAME, - when(col("add.path").isNotNull, canonicalPath(col("add.path")))) - .withColumn( - REMOVE_PATH_CANONICAL_COL_NAME, - when(col("remove.path").isNotNull, canonicalPath(col("remove.path")))) + .withColumn(ADD_PATH_CANONICAL_COL_NAME, when( + col("add.path").isNotNull, canonicalPath(col("add.path")))) + .withColumn(REMOVE_PATH_CANONICAL_COL_NAME, when( + col("remove.path").isNotNull, canonicalPath(col("remove.path")))) .repartition( getNumPartitions, coalesce(col(ADD_PATH_CANONICAL_COL_NAME), col(REMOVE_PATH_CANONICAL_COL_NAME))) .sortWithinPartitions(ACTION_SORT_COL_NAME) - .withColumn( - "add", - when( - col("add.path").isNotNull, - struct( - col(ADD_PATH_CANONICAL_COL_NAME).as("path"), - col("add.partitionValues"), - col("add.size"), - col("add.modificationTime"), - col("add.dataChange"), - col(ADD_STATS_TO_USE_COL_NAME).as("stats"), - col("add.tags") - ) - ) - ) - .withColumn( - "remove", - when( - col("remove.path").isNotNull, - col("remove").withField("path", col(REMOVE_PATH_CANONICAL_COL_NAME)))) + .withColumn("add", when( + col("add.path").isNotNull, + struct( + col(ADD_PATH_CANONICAL_COL_NAME).as("path"), + col("add.partitionValues"), + col("add.size"), + col("add.modificationTime"), + col("add.dataChange"), + col(ADD_STATS_TO_USE_COL_NAME).as("stats"), + col("add.tags"), + col("add.deletionVector") + ))) + .withColumn("remove", when( + col("remove.path").isNotNull, + col("remove").withField("path", col(REMOVE_PATH_CANONICAL_COL_NAME)))) .as[SingleAction] - .mapPartitions { - iter => - val state: LogReplay = - new InMemoryLogReplay( - localMinFileRetentionTimestamp, - localMinSetTransactionRetentionTimestamp) - state.append(0, iter.map(_.unwrap)) - state.checkpoint.map(_.wrap) + .mapPartitions { iter => + val state: LogReplay = + new InMemoryLogReplay( + localMinFileRetentionTimestamp, + localMinSetTransactionRetentionTimestamp) + state.append(0, iter.map(_.unwrap)) + state.checkpoint.map(_.wrap) } } } + /** + * Pulls the protocol and metadata of the table from the files that are used to compute the + * Snapshot directly--without triggering a full state reconstruction. This is important, because + * state reconstruction depends on protocol and metadata for correctness. + */ + protected def protocolAndMetadataReconstruction(): Array[(Protocol, Metadata)] = { + import implicits._ + + val schemaToUse = Action.logSchema(Set("protocol", "metaData")) + fileIndices.map(deltaLog.loadIndex(_, schemaToUse)) + .reduceOption(_.union(_)).getOrElse(emptyDF) + .withColumn(ACTION_SORT_COL_NAME, input_file_name()) + .select("protocol", "metaData", ACTION_SORT_COL_NAME) + .where("protocol.minReaderVersion is not null or metaData.id is not null") + .as[(Protocol, Metadata, String)] + .collect() + .sortBy(_._3) + .map { case (p, m, _) => p -> m } + } + def redactedPath: String = Utils.redact(spark.sessionState.conf.stringRedactionPattern, path.toUri.toString) @@ -189,7 +205,9 @@ class Snapshot( cachedState.getDF } - /** A Map of alias to aggregations which needs to be done to calculate the `computedState` */ + /** + * A Map of alias to aggregations which needs to be done to calculate the `computedState` + */ protected def aggregationsToComputeState: Map[String, Column] = { Map( // sum may return null for empty data set. @@ -223,24 +241,75 @@ class Snapshot( recordDeltaEvent( deltaLog, opType = "delta.assertions.missingAction", - data = - Map("version" -> version.toString, "action" -> "Protocol", "source" -> "Snapshot")) + data = Map( + "version" -> version.toString, "action" -> "Protocol", "source" -> "Snapshot")) + throw DeltaErrors.actionNotFoundException("protocol", version) + } else if (_computedState.protocol != protocol) { + recordDeltaEvent( + deltaLog, + opType = "delta.assertions.mismatchedAction", + data = Map( + "version" -> version.toString, "action" -> "Protocol", "source" -> "Snapshot", + "computedState.protocol" -> _computedState.protocol, + "extracted.protocol" -> protocol)) throw DeltaErrors.actionNotFoundException("protocol", version) } + if (_computedState.metadata == null) { recordDeltaEvent( deltaLog, opType = "delta.assertions.missingAction", - data = - Map("version" -> version.toString, "action" -> "Metadata", "source" -> "Metadata")) + data = Map( + "version" -> version.toString, "action" -> "Metadata", "source" -> "Metadata")) + throw DeltaErrors.actionNotFoundException("metadata", version) + } else if (_computedState.metadata != metadata) { + recordDeltaEvent( + deltaLog, + opType = "delta.assertions.mismatchedAction", + data = Map( + "version" -> version.toString, "action" -> "Metadata", "source" -> "Snapshot", + "computedState.metadata" -> _computedState.metadata, + "extracted.metadata" -> metadata)) throw DeltaErrors.actionNotFoundException("metadata", version) - } else { - _computedState } + + _computedState } } } + // Used by [[protocol]] and [[metadata]] below + private lazy val (_protocol, _metadata): (Protocol, Metadata) = { + // Should be small. At most 'checkpointInterval' rows, unless new commits are coming + // in before a checkpoint can be written + var protocol: Protocol = null + var metadata: Metadata = null + protocolAndMetadataReconstruction().foreach { + case (p: Protocol, _) => protocol = p + case (_, m: Metadata) => metadata = m + } + + if (protocol == null) { + recordDeltaEvent( + deltaLog, + opType = "delta.assertions.missingAction", + data = Map( + "version" -> version.toString, "action" -> "Protocol", "source" -> "Snapshot")) + throw DeltaErrors.actionNotFoundException("protocol", version) + } + + if (metadata == null) { + recordDeltaEvent( + deltaLog, + opType = "delta.assertions.missingAction", + data = Map( + "version" -> version.toString, "action" -> "Metadata", "source" -> "Snapshot")) + throw DeltaErrors.actionNotFoundException("metadata", version) + } + + protocol -> metadata + } + def sizeInBytes: Long = computedState.sizeInBytes def numOfSetTransactions: Long = computedState.numOfSetTransactions def numOfFiles: Long = computedState.numOfFiles @@ -248,18 +317,34 @@ class Snapshot( def numOfMetadata: Long = computedState.numOfMetadata def numOfProtocol: Long = computedState.numOfProtocol def setTransactions: Seq[SetTransaction] = computedState.setTransactions - override def metadata: Metadata = computedState.metadata - override def protocol: Protocol = computedState.protocol + override def metadata: Metadata = _metadata + override def protocol: Protocol = _protocol def fileSizeHistogram: Option[FileSizeHistogram] = computedState.fileSizeHistogram - private[delta] def sizeInBytesOpt: Option[Long] = Some(sizeInBytes) - private[delta] def setTransactionsOpt: Option[Seq[SetTransaction]] = Some(setTransactions) - private[delta] def numOfFilesOpt: Option[Long] = Some(numOfFiles) + private[delta] def sizeInBytesIfKnown: Option[Long] = Some(sizeInBytes) + private[delta] def setTransactionsIfKnown: Option[Seq[SetTransaction]] = Some(setTransactions) + private[delta] def numOfFilesIfKnown: Option[Long] = Some(numOfFiles) + + /** + * Tombstones before the [[minFileRetentionTimestamp]] timestamp will be dropped from the + * checkpoint. + */ + private[delta] def minFileRetentionTimestamp: Long = { + deltaLog.clock.getTimeMillis() - DeltaLog.tombstoneRetentionMillis(metadata) + } + + /** + * [[SetTransaction]]s before [[minSetTransactionRetentionTimestamp]] will be considered expired + * and dropped from the snapshot. + */ + private[delta] def minSetTransactionRetentionTimestamp: Option[Long] = { + DeltaLog.minSetTransactionRetentionInterval(metadata).map(deltaLog.clock.getTimeMillis() - _) + } /** - * Computes all the information that is needed by the checksum for the current snapshot. May kick - * off state reconstruction if needed by any of the underlying fields. Note that it's safe to set - * txnId to none, since the snapshot doesn't always have a txn attached. E.g. if a snapshot is - * created by reading a checkpoint, then no txnId is present. + * Computes all the information that is needed by the checksum for the current snapshot. + * May kick off state reconstruction if needed by any of the underlying fields. + * Note that it's safe to set txnId to none, since the snapshot doesn't always have a txn + * attached. E.g. if a snapshot is created by reading a checkpoint, then no txnId is present. */ def computeChecksum: VersionChecksum = VersionChecksum( txnId = None, @@ -271,8 +356,7 @@ class Snapshot( metadata = metadata, protocol = protocol, histogramOpt = fileSizeHistogram, - allFiles = checksumOpt.flatMap(_.allFiles) - ) + allFiles = checksumOpt.flatMap(_.allFiles)) /** A map to look up transaction version by appId. */ lazy val transactions: Map[String, Long] = setTransactions.map(t => t.appId -> t.version).toMap @@ -300,17 +384,23 @@ class Snapshot( lazy val numIndexedCols: Int = DeltaConfigs.DATA_SKIPPING_NUM_INDEXED_COLS.fromMetaData(metadata) /** Return the set of properties of the table. */ - def getProperties: mutable.HashMap[String, String] = { - val base = new mutable.HashMap[String, String]() - metadata.configuration.foreach { - case (k, v) => - if (k != "path") { - base.put(k, v) - } + def getProperties: mutable.Map[String, String] = { + val base = new mutable.LinkedHashMap[String, String]() + metadata.configuration.foreach { case (k, v) => + if (k != "path") { + base.put(k, v) + } } base.put(Protocol.MIN_READER_VERSION_PROP, protocol.minReaderVersion.toString) base.put(Protocol.MIN_WRITER_VERSION_PROP, protocol.minWriterVersion.toString) - base + if (protocol.supportsReaderFeatures || protocol.supportsWriterFeatures) { + val features = protocol.readerAndWriterFeatureNames.map(name => + s"${TableFeatureProtocolUtils.FEATURE_PROP_PREFIX}$name" -> + TableFeatureProtocolUtils.FEATURE_PROP_SUPPORTED) + base ++ features.toSeq.sorted + } else { + base + } } // Given the list of files from `LogSegment`, create respective file indices to help create @@ -345,10 +435,8 @@ class Snapshot( * config settings for delta.checkpoint.writeStatsAsJson and delta.checkpoint.writeStatsAsStruct). */ protected def loadActions: DataFrame = { - val dfs = fileIndices.map(index => Dataset.ofRows(spark, deltaLog.indexToRelation(index))) - dfs - .reduceOption(_.union(_)) - .getOrElse(emptyDF) + fileIndices.map(deltaLog.loadIndex(_)) + .reduceOption(_.union(_)).getOrElse(emptyDF) .withColumn(ACTION_SORT_COL_NAME, input_file_name()) .withColumn(ADD_STATS_TO_USE_COL_NAME, col("add.stats")) } @@ -356,6 +444,7 @@ class Snapshot( protected def emptyDF: DataFrame = spark.createDataFrame(spark.sparkContext.emptyRDD[Row], logSchema) + override def logInfo(msg: => String): Unit = { super.logInfo(s"[tableId=${deltaLog.tableId}] " + msg) } @@ -380,21 +469,22 @@ class Snapshot( s"${getClass.getSimpleName}(path=$path, version=$version, metadata=$metadata, " + s"logSegment=$logSegment, checksumOpt=$checksumOpt)" - override def filesForScan(filters: Seq[Expression], keepNumRecords: Boolean): DeltaScan = { + // --- modified start + override def filesForScan(limit: Long): DeltaScan = { val deltaScan = ClickhouseSnapshot.deltaScanCache.get( - FilterExprsAsKey(path, ClickhouseSnapshot.genSnapshotId(this), filters, None), + FilterExprsAsKey(path, ClickhouseSnapshot.genSnapshotId(this), Seq.empty, Some(limit)), () => { - super.filesForScan(filters, keepNumRecords) + super.filesForScan(limit) }) replaceWithAddMergeTreeParts(deltaScan) } - override def filesForScan(limit: Long): DeltaScan = { + override def filesForScan(filters: Seq[Expression], keepNumRecords: Boolean): DeltaScan = { val deltaScan = ClickhouseSnapshot.deltaScanCache.get( - FilterExprsAsKey(path, ClickhouseSnapshot.genSnapshotId(this), Seq.empty, Some(limit)), + FilterExprsAsKey(path, ClickhouseSnapshot.genSnapshotId(this), filters, None), () => { - super.filesForScan(limit) + super.filesForScan(filters, keepNumRecords) }) replaceWithAddMergeTreeParts(deltaScan) @@ -411,30 +501,35 @@ class Snapshot( } private def replaceWithAddMergeTreeParts(deltaScan: DeltaScan) = { - DeltaScan.apply( - deltaScan.version, - deltaScan.files - .map( - addFile => { - val addFileAsKey = AddFileAsKey(addFile) - - val ret = ClickhouseSnapshot.addFileToAddMTPCache.get(addFileAsKey) - // this is for later use - ClickhouseSnapshot.pathToAddMTPCache.put(ret.fullPartPath(), ret) - ret - }), - deltaScan.total, - deltaScan.partition, - deltaScan.scanned - )( - deltaScan.scannedSnapshot, - deltaScan.partitionFilters, - deltaScan.dataFilters, - deltaScan.unusedFilters, - deltaScan.scanDurationMs, - deltaScan.dataSkippingType - ) + if (ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { + DeltaScan.apply( + deltaScan.version, + deltaScan.files + .map( + addFile => { + val addFileAsKey = AddFileAsKey(addFile) + + val ret = ClickhouseSnapshot.addFileToAddMTPCache.get(addFileAsKey) + // this is for later use + ClickhouseSnapshot.pathToAddMTPCache.put(ret.fullPartPath(), ret) + ret + }), + deltaScan.total, + deltaScan.partition, + deltaScan.scanned + )( + deltaScan.scannedSnapshot, + deltaScan.partitionFilters, + deltaScan.dataFilters, + deltaScan.unusedFilters, + deltaScan.scanDurationMs, + deltaScan.dataSkippingType + ) + } else { + deltaScan + } } + // --- modified end logInfo(s"Created snapshot $this") init() @@ -450,64 +545,51 @@ object Snapshot extends DeltaLogging { /** Verifies that a set of delta or checkpoint files to be read actually belongs to this table. */ private def assertLogFilesBelongToTable(logBasePath: Path, files: Seq[FileStatus]): Unit = { - files.map(_.getPath).foreach { - filePath => - if (new Path(filePath.toUri).getParent != new Path(logBasePath.toUri)) { - // scalastyle:off throwerror - throw new AssertionError( - s"File ($filePath) doesn't belong in the " + - s"transaction log at $logBasePath. Please contact Databricks Support.") - // scalastyle:on throwerror - } + files.map(_.getPath).foreach { filePath => + if (new Path(filePath.toUri).getParent != new Path(logBasePath.toUri)) { + // scalastyle:off throwerror + throw new AssertionError(s"File ($filePath) doesn't belong in the " + + s"transaction log at $logBasePath. Please contact Databricks Support.") + // scalastyle:on throwerror + } } } /** * Metrics and metadata computed around the Delta table. - * @param sizeInBytes - * The total size of the table (of active files, not including tombstones). - * @param numOfSetTransactions - * Number of streams writing to this table. - * @param numOfFiles - * The number of files in this table. - * @param numOfRemoves - * The number of tombstones in the state. - * @param numOfMetadata - * The number of metadata actions in the state. Should be 1. - * @param numOfProtocol - * The number of protocol actions in the state. Should be 1. - * @param setTransactions - * The streaming queries writing to this table. - * @param metadata - * The metadata of the table. - * @param protocol - * The protocol version of the Delta table. - * @param fileSizeHistogram - * A Histogram class tracking the file counts and total bytes in different size ranges. + * @param sizeInBytes The total size of the table (of active files, not including tombstones). + * @param numOfSetTransactions Number of streams writing to this table. + * @param numOfFiles The number of files in this table. + * @param numOfRemoves The number of tombstones in the state. + * @param numOfMetadata The number of metadata actions in the state. Should be 1. + * @param numOfProtocol The number of protocol actions in the state. Should be 1. + * @param setTransactions The streaming queries writing to this table. + * @param metadata The metadata of the table. + * @param protocol The protocol version of the Delta table. + * @param fileSizeHistogram A Histogram class tracking the file counts and total bytes + * in different size ranges. */ case class State( - sizeInBytes: Long, - numOfSetTransactions: Long, - numOfFiles: Long, - numOfRemoves: Long, - numOfMetadata: Long, - numOfProtocol: Long, - setTransactions: Seq[SetTransaction], - metadata: Metadata, - protocol: Protocol, - fileSizeHistogram: Option[FileSizeHistogram] = None) + sizeInBytes: Long, + numOfSetTransactions: Long, + numOfFiles: Long, + numOfRemoves: Long, + numOfMetadata: Long, + numOfProtocol: Long, + setTransactions: Seq[SetTransaction], + metadata: Metadata, + protocol: Protocol, + fileSizeHistogram: Option[FileSizeHistogram] = None + ) } /** * An initial snapshot with only metadata specified. Useful for creating a DataFrame from an * existing parquet table during its conversion to delta. * - * @param logPath - * the path to transaction log - * @param deltaLog - * the delta log object - * @param metadata - * the metadata of the table + * @param logPath the path to transaction log + * @param deltaLog the delta log object + * @param metadata the metadata of the table */ class InitialSnapshot( val logPath: Path, @@ -517,27 +599,30 @@ class InitialSnapshot( path = logPath, version = -1, logSegment = LogSegment.empty(logPath), - minFileRetentionTimestamp = -1, deltaLog = deltaLog, timestamp = -1, - checksumOpt = None, - minSetTransactionRetentionTimestamp = None + checksumOpt = None ) { def this(logPath: Path, deltaLog: DeltaLog) = this( logPath, deltaLog, Metadata( - configuration = - DeltaConfigs.mergeGlobalConfigs(SparkSession.active.sessionState.conf, Map.empty), - createdTime = Some(System.currentTimeMillis())) - ) + configuration = DeltaConfigs.mergeGlobalConfigs( + sqlConfs = SparkSession.active.sessionState.conf, + tableConf = Map.empty, + ignoreProtocolConfsOpt = Some( + DeltaConfigs.ignoreProtocolDefaultsIsSet( + sqlConfs = SparkSession.active.sessionState.conf, + tableConf = deltaLog.allOptions))), + createdTime = Some(System.currentTimeMillis()))) override def stateDS: Dataset[SingleAction] = emptyDF.as[SingleAction] override def stateDF: DataFrame = emptyDF override protected lazy val computedState: Snapshot.State = initialState + override def protocol: Protocol = computedState.protocol private def initialState: Snapshot.State = { - val protocol = Protocol.forNewTable(spark, metadata) + val protocol = Protocol.forNewTable(spark, Some(metadata)) Snapshot.State( sizeInBytes = 0L, numOfSetTransactions = 0L, @@ -550,5 +635,4 @@ class InitialSnapshot( protocol = protocol ) } - } diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/DeleteCommand.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/DeleteCommand.scala similarity index 61% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/DeleteCommand.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/DeleteCommand.scala index 006a3fce8429..5f9c2953ba16 100644 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/DeleteCommand.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/DeleteCommand.scala @@ -14,33 +14,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.delta.commands -import org.apache.spark.SparkContext -import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, EqualNullSafe, Expression, If, Literal, Not} -import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.{DeltaDelete, LogicalPlan} import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions.{Action, AddCDCFile, AddFile, FileAction} import org.apache.spark.sql.delta.commands.DeleteCommand.{rewritingFilesMsg, FINDING_TOUCHED_FILES_MSG} import org.apache.spark.sql.delta.commands.MergeIntoCommand.totalBytesAndDistinctPartitionValues import org.apache.spark.sql.delta.files.TahoeBatchFileIndex import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.delta.util.Utils +import com.fasterxml.jackson.databind.annotation.JsonDeserialize + +import org.apache.spark.SparkContext +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, EqualNullSafe, Expression, If, Literal, Not} +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.{DeltaDelete, LogicalPlan} import org.apache.spark.sql.execution.command.LeafRunnableCommand -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.metric.SQLMetrics.{createMetric, createTimingMetric} import org.apache.spark.sql.functions.{col, explode, input_file_name, split} import org.apache.spark.sql.types.LongType -import com.fasterxml.jackson.databind.annotation.JsonDeserialize - /** * Gluten overwrite Delta: * - * This file is copied from Delta 2.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 2.3.0. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). */ @@ -60,8 +61,8 @@ trait DeleteCommandMetrics { self: LeafRunnableCommand => "numPartitionsAddedTo" -> createMetric(sc, "number of partitions added"), "numPartitionsRemovedFrom" -> createMetric(sc, "number of partitions removed"), "numCopiedRows" -> createMetric(sc, "number of rows copied"), - "numBytesAdded" -> createMetric(sc, "number of bytes added"), - "numBytesRemoved" -> createMetric(sc, "number of bytes removed"), + "numAddedBytes" -> createMetric(sc, "number of bytes added"), + "numRemovedBytes" -> createMetric(sc, "number of bytes removed"), "executionTimeMs" -> createTimingMetric(sc, "time taken to execute the entire operation"), "scanTimeMs" -> @@ -73,7 +74,7 @@ trait DeleteCommandMetrics { self: LeafRunnableCommand => "numTouchedRows" -> createMetric(sc, "number of rows touched") ) - def getDeletedRowsFromAddFilesAndUpdateMetrics(files: Seq[AddFile]): Option[Long] = { + def getDeletedRowsFromAddFilesAndUpdateMetrics(files: Seq[AddFile]) : Option[Long] = { if (!conf.getConf(DeltaSQLConf.DELTA_DML_METRICS_FROM_METADATA)) { return None; } @@ -97,15 +98,18 @@ trait DeleteCommandMetrics { self: LeafRunnableCommand => /** * Performs a Delete based on the search condition * - * Algorithm: 1) Scan all the files and determine which files have the rows that need to be deleted. - * 2) Traverse the affected files and rebuild the touched files. 3) Use the Delta protocol to - * atomically write the remaining rows to new files and remove the affected files that are - * identified in step 1. + * Algorithm: + * 1) Scan all the files and determine which files have + * the rows that need to be deleted. + * 2) Traverse the affected files and rebuild the touched files. + * 3) Use the Delta protocol to atomically write the remaining rows to new files and remove + * the affected files that are identified in step 1. */ -case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Option[Expression]) - extends LeafRunnableCommand - with DeltaCommand - with DeleteCommandMetrics { +case class DeleteCommand( + deltaLog: DeltaLog, + target: LogicalPlan, + condition: Option[Expression]) + extends LeafRunnableCommand with DeltaCommand with DeleteCommandMetrics { override def innerChildren: Seq[QueryPlan[_]] = Seq(target) @@ -115,13 +119,15 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt final override def run(sparkSession: SparkSession): Seq[Row] = { recordDeltaOperation(deltaLog, "delta.dml.delete") { - deltaLog.assertRemovable() - deltaLog.withNewTransaction { - txn => - val deleteActions = performDelete(sparkSession, deltaLog, txn) - if (deleteActions.nonEmpty) { - txn.commit(deleteActions, DeltaOperations.Delete(condition.map(_.sql).toSeq)) - } + deltaLog.withNewTransaction { txn => + DeltaLog.assertRemovable(txn.snapshot) + if (hasBeenExecuted(txn, sparkSession)) { + sendDriverMetrics(sparkSession, metrics) + return Seq.empty + } + + val deleteActions = performDelete(sparkSession, deltaLog, txn) + txn.commitIfNeeded(deleteActions, DeltaOperations.Delete(condition.map(_.sql).toSeq)) } // Re-cache all cached plans(including this relation itself, if it's cached) that refer to // this data source relation. @@ -150,9 +156,9 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt var numAddedChangeFiles: Long = 0 var scanTimeMs: Long = 0 var rewriteTimeMs: Long = 0 - var numBytesAdded: Long = 0 + var numAddedBytes: Long = 0 var changeFileBytes: Long = 0 - var numBytesRemoved: Long = 0 + var numRemovedBytes: Long = 0 var numFilesBeforeSkipping: Long = 0 var numBytesBeforeSkipping: Long = 0 var numFilesAfterSkipping: Long = 0 @@ -175,7 +181,7 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt numRemovedFiles = allFiles.size scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 val (numBytes, numPartitions) = totalBytesAndDistinctPartitionValues(allFiles) - numBytesRemoved = numBytes + numRemovedBytes = numBytes numFilesBeforeSkipping = numRemovedFiles numBytesBeforeSkipping = numBytes numFilesAfterSkipping = numRemovedFiles @@ -192,9 +198,7 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt case Some(cond) => val (metadataPredicates, otherPredicates) = DeltaTableUtils.splitMetadataAndDataPredicates( - cond, - txn.metadata.partitionColumns, - sparkSession) + cond, txn.metadata.partitionColumns, sparkSession) numFilesBeforeSkipping = txn.snapshot.numOfFiles numBytesBeforeSkipping = txn.snapshot.sizeInBytes @@ -209,7 +213,7 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 numRemovedFiles = candidateFiles.size - numBytesRemoved = candidateFiles.map(_.size).sum + numRemovedBytes = candidateFiles.map(_.size).sum numFilesAfterSkipping = candidateFiles.size val (numCandidateBytes, numCandidatePartitions) = totalBytesAndDistinctPartitionValues(candidateFiles) @@ -224,7 +228,15 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt candidateFiles.map(_.removeWithTimestamp(operationTimestamp)) } else { // Case 3: Delete the rows based on the condition. - val candidateFiles = txn.filterFiles(metadataPredicates ++ otherPredicates) + + // Should we write the DVs to represent the deleted rows? + val shouldWriteDVs = shouldWritePersistentDeletionVectors(sparkSession, txn) + + val candidateFiles = txn.filterFiles( + metadataPredicates ++ otherPredicates, + keepNumRecords = shouldWriteDVs) + // `candidateFiles` contains the files filtered using statistics and delete condition + // They may or may not contains any rows that need to be deleted. numFilesAfterSkipping = candidateFiles.size val (numCandidateBytes, numCandidatePartitions) = @@ -237,89 +249,104 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt val nameToAddFileMap = generateCandidateFileMap(deltaLog.dataPath, candidateFiles) val fileIndex = new TahoeBatchFileIndex( - sparkSession, - "delete", - candidateFiles, - deltaLog, - deltaLog.dataPath, - txn.snapshot) - // Keep everything from the resolved target except a new TahoeFileIndex - // that only involves the affected files instead of all files. - val newTarget = DeltaTableUtils.replaceFileIndex(target, fileIndex) - val data = Dataset.ofRows(sparkSession, newTarget) - val deletedRowCount = metrics("numDeletedRows") - val deletedRowUdf = DeltaUDF - .boolean { - () => - deletedRowCount += 1 - true - } - .asNondeterministic() - val filesToRewrite = - withStatusCode("DELTA", FINDING_TOUCHED_FILES_MSG) { - if (candidateFiles.isEmpty) { - Array.empty[String] - } else { - data - .filter(new Column(cond)) - .select(input_file_name().as("input_files")) - .filter(deletedRowUdf()) - .select(explode(split(col("input_files"), ","))) - .distinct() - .as[String] - .collect() - } - } + sparkSession, "delete", candidateFiles, deltaLog, deltaLog.dataPath, txn.snapshot) + if (shouldWriteDVs) { + val targetDf = DeleteWithDeletionVectorsHelper.createTargetDfForScanningForMatches( + sparkSession, + target, + fileIndex) - numRemovedFiles = filesToRewrite.length - scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 - if (filesToRewrite.isEmpty) { - // Case 3.1: no row matches and no delete will be triggered - if (txn.metadata.partitionColumns.nonEmpty) { - numPartitionsRemovedFrom = Some(0) - numPartitionsAddedTo = Some(0) - } - Nil - } else { - // Case 3.2: some files need an update to remove the deleted files - // Do the second pass and just read the affected files - val baseRelation = buildBaseRelation( + // Does the target table already has DVs enabled? If so, we need to read the table + // with deletion vectors. + val mustReadDeletionVectors = DeletionVectorUtils.deletionVectorsReadable(txn.snapshot) + + val touchedFiles = DeleteWithDeletionVectorsHelper.findTouchedFiles( sparkSession, txn, - "delete", - deltaLog.dataPath, - filesToRewrite, - nameToAddFileMap) + mustReadDeletionVectors, + deltaLog, + targetDf, + fileIndex, + cond) + + if (touchedFiles.nonEmpty) { + DeleteWithDeletionVectorsHelper.processUnmodifiedData(touchedFiles) + } else { + Nil // Nothing to update + } + } else { // Keep everything from the resolved target except a new TahoeFileIndex // that only involves the affected files instead of all files. - val newTarget = DeltaTableUtils.replaceFileIndex(target, baseRelation.location) - val targetDF = Dataset.ofRows(sparkSession, newTarget) - val filterCond = Not(EqualNullSafe(cond, Literal.TrueLiteral)) - val rewrittenActions = rewriteFiles(txn, targetDF, filterCond, filesToRewrite.length) - val (changeFiles, rewrittenFiles) = rewrittenActions - .partition(_.isInstanceOf[AddCDCFile]) - numAddedFiles = rewrittenFiles.size - val removedFiles = - filesToRewrite.map(f => getTouchedFile(deltaLog.dataPath, f, nameToAddFileMap)) - val (removedBytes, removedPartitions) = - totalBytesAndDistinctPartitionValues(removedFiles) - numBytesRemoved = removedBytes - val (rewrittenBytes, rewrittenPartitions) = - totalBytesAndDistinctPartitionValues(rewrittenFiles) - numBytesAdded = rewrittenBytes - if (txn.metadata.partitionColumns.nonEmpty) { - numPartitionsRemovedFrom = Some(removedPartitions) - numPartitionsAddedTo = Some(rewrittenPartitions) + val newTarget = DeltaTableUtils.replaceFileIndex(target, fileIndex) + val data = Dataset.ofRows(sparkSession, newTarget) + val deletedRowCount = metrics("numDeletedRows") + val deletedRowUdf = DeltaUDF.boolean { () => + deletedRowCount += 1 + true + }.asNondeterministic() + val filesToRewrite = + withStatusCode("DELTA", FINDING_TOUCHED_FILES_MSG) { + if (candidateFiles.isEmpty) { + Array.empty[String] + } else { + // --- modified start + data.filter(new Column(cond)) + .select(input_file_name().as("input_files")) + .filter(deletedRowUdf()) + .select(explode(split(col("input_files"), ","))) + .distinct() + .as[String] + .collect() + // --- modified end + } + } + + numRemovedFiles = filesToRewrite.length + scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + if (filesToRewrite.isEmpty) { + // Case 3.1: no row matches and no delete will be triggered + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsRemovedFrom = Some(0) + numPartitionsAddedTo = Some(0) + } + Nil + } else { + // Case 3.2: some files need an update to remove the deleted files + // Do the second pass and just read the affected files + val baseRelation = buildBaseRelation( + sparkSession, txn, "delete", deltaLog.dataPath, filesToRewrite, nameToAddFileMap) + // Keep everything from the resolved target except a new TahoeFileIndex + // that only involves the affected files instead of all files. + val newTarget = DeltaTableUtils.replaceFileIndex(target, baseRelation.location) + val targetDF = Dataset.ofRows(sparkSession, newTarget) + val filterCond = Not(EqualNullSafe(cond, Literal.TrueLiteral)) + val rewrittenActions = rewriteFiles(txn, targetDF, filterCond, filesToRewrite.length) + val (changeFiles, rewrittenFiles) = rewrittenActions + .partition(_.isInstanceOf[AddCDCFile]) + numAddedFiles = rewrittenFiles.size + val removedFiles = filesToRewrite.map(f => + getTouchedFile(deltaLog.dataPath, f, nameToAddFileMap)) + val (removedBytes, removedPartitions) = + totalBytesAndDistinctPartitionValues(removedFiles) + numRemovedBytes = removedBytes + val (rewrittenBytes, rewrittenPartitions) = + totalBytesAndDistinctPartitionValues(rewrittenFiles) + numAddedBytes = rewrittenBytes + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsRemovedFrom = Some(removedPartitions) + numPartitionsAddedTo = Some(rewrittenPartitions) + } + numAddedChangeFiles = changeFiles.size + changeFileBytes = changeFiles.collect { case f: AddCDCFile => f.size }.sum + rewriteTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 - scanTimeMs + numDeletedRows = Some(metrics("numDeletedRows").value) + numCopiedRows = + Some(metrics("numTouchedRows").value - metrics("numDeletedRows").value) + + val operationTimestamp = System.currentTimeMillis() + removeFilesFromPaths( + deltaLog, nameToAddFileMap, filesToRewrite, operationTimestamp) ++ rewrittenActions } - numAddedChangeFiles = changeFiles.size - changeFileBytes = changeFiles.collect { case f: AddCDCFile => f.size }.sum - rewriteTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 - scanTimeMs - numDeletedRows = Some(metrics("numDeletedRows").value) - numCopiedRows = Some(metrics("numTouchedRows").value - metrics("numDeletedRows").value) - - val operationTimestamp = System.currentTimeMillis() - removeFilesFromPaths(deltaLog, nameToAddFileMap, filesToRewrite, operationTimestamp) ++ - rewrittenActions } } } @@ -331,8 +358,8 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt metrics("rewriteTimeMs").set(rewriteTimeMs) metrics("numAddedChangeFiles").set(numAddedChangeFiles) metrics("changeFileBytes").set(changeFileBytes) - metrics("numBytesAdded").set(numBytesAdded) - metrics("numBytesRemoved").set(numBytesRemoved) + metrics("numAddedBytes").set(numAddedBytes) + metrics("numRemovedBytes").set(numRemovedBytes) metrics("numFilesBeforeSkipping").set(numFilesBeforeSkipping) metrics("numBytesBeforeSkipping").set(numBytesBeforeSkipping) metrics("numFilesAfterSkipping").set(numFilesAfterSkipping) @@ -342,9 +369,7 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt numPartitionsRemovedFrom.foreach(metrics("numPartitionsRemovedFrom").set) numCopiedRows.foreach(metrics("numCopiedRows").set) txn.registerSQLMetrics(sparkSession, metrics) - // This is needed to make the SQL metrics visible in the Spark UI - val executionId = sparkSession.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sparkSession.sparkContext, executionId, metrics.values.toSeq) + sendDriverMetrics(sparkSession, metrics) recordDeltaEvent( deltaLog, @@ -366,18 +391,23 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt numPartitionsRemovedFrom, numCopiedRows, numDeletedRows, - numBytesAdded, - numBytesRemoved, + numAddedBytes, + numRemovedBytes, changeFileBytes = changeFileBytes, scanTimeMs, - rewriteTimeMs - ) + rewriteTimeMs) ) - deleteActions + if (deleteActions.nonEmpty) { + createSetTransaction(sparkSession, deltaLog).toSeq ++ deleteActions + } else { + Seq.empty + } } - /** Returns the list of [[AddFile]]s and [[AddCDCFile]]s that have been re-written. */ + /** + * Returns the list of [[AddFile]]s and [[AddCDCFile]]s that have been re-written. + */ private def rewriteFiles( txn: OptimisticTransaction, baseData: DataFrame, @@ -387,15 +417,13 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt // number of total rows that we have seen / are either copying or deleting (sum of both). val numTouchedRows = metrics("numTouchedRows") - val numTouchedRowsUdf = DeltaUDF - .boolean { - () => - numTouchedRows += 1 - true - } - .asNondeterministic() + val numTouchedRowsUdf = DeltaUDF.boolean { () => + numTouchedRows += 1 + true + }.asNondeterministic() - withStatusCode("DELTA", rewritingFilesMsg(numFilesToRewrite)) { + withStatusCode( + "DELTA", rewritingFilesMsg(numFilesToRewrite)) { val dfToWrite = if (shouldWriteCdc) { import org.apache.spark.sql.delta.commands.cdc.CDCReader._ // The logic here ends up being surprisingly elegant, with all source rows ending up in @@ -418,6 +446,14 @@ case class DeleteCommand(deltaLog: DeltaLog, target: LogicalPlan, condition: Opt txn.writeFiles(dfToWrite) } } + + def shouldWritePersistentDeletionVectors( + spark: SparkSession, txn: OptimisticTransaction): Boolean = { + // DELETE with DVs only enabled for tests. + Utils.isTesting && + spark.conf.get(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS) && + DeletionVectorUtils.deletionVectorsWritable(txn.snapshot) + } } object DeleteCommand { @@ -441,51 +477,29 @@ object DeleteCommand { /** * Used to report details about delete. * - * @param condition: - * what was the delete condition - * @param numFilesTotal: - * how big is the table - * @param numTouchedFiles: - * how many files did we touch. Alias for `numFilesAfterSkipping` - * @param numRewrittenFiles: - * how many files had to be rewritten. Alias for `numAddedFiles` - * @param numRemovedFiles: - * how many files we removed. Alias for `numTouchedFiles` - * @param numAddedFiles: - * how many files we added. Alias for `numRewrittenFiles` - * @param numAddedChangeFiles: - * how many change files were generated - * @param numFilesBeforeSkipping: - * how many candidate files before skipping - * @param numBytesBeforeSkipping: - * how many candidate bytes before skipping - * @param numFilesAfterSkipping: - * how many candidate files after skipping - * @param numBytesAfterSkipping: - * how many candidate bytes after skipping - * @param numPartitionsAfterSkipping: - * how many candidate partitions after skipping - * @param numPartitionsAddedTo: - * how many new partitions were added - * @param numPartitionsRemovedFrom: - * how many partitions were removed - * @param numCopiedRows: - * how many rows were copied - * @param numDeletedRows: - * how many rows were deleted - * @param numBytesAdded: - * how many bytes were added - * @param numBytesRemoved: - * how many bytes were removed - * @param changeFileBytes: - * total size of change files generated - * @param scanTimeMs: - * how long did finding take - * @param rewriteTimeMs: - * how long did rewriting take + * @param condition: what was the delete condition + * @param numFilesTotal: how big is the table + * @param numTouchedFiles: how many files did we touch. Alias for `numFilesAfterSkipping` + * @param numRewrittenFiles: how many files had to be rewritten. Alias for `numAddedFiles` + * @param numRemovedFiles: how many files we removed. Alias for `numTouchedFiles` + * @param numAddedFiles: how many files we added. Alias for `numRewrittenFiles` + * @param numAddedChangeFiles: how many change files were generated + * @param numFilesBeforeSkipping: how many candidate files before skipping + * @param numBytesBeforeSkipping: how many candidate bytes before skipping + * @param numFilesAfterSkipping: how many candidate files after skipping + * @param numBytesAfterSkipping: how many candidate bytes after skipping + * @param numPartitionsAfterSkipping: how many candidate partitions after skipping + * @param numPartitionsAddedTo: how many new partitions were added + * @param numPartitionsRemovedFrom: how many partitions were removed + * @param numCopiedRows: how many rows were copied + * @param numDeletedRows: how many rows were deleted + * @param numBytesAdded: how many bytes were added + * @param numBytesRemoved: how many bytes were removed + * @param changeFileBytes: total size of change files generated + * @param scanTimeMs: how long did finding take + * @param rewriteTimeMs: how long did rewriting take * - * @note - * All the time units are milliseconds. + * @note All the time units are milliseconds. */ case class DeleteMetric( condition: String, diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala similarity index 70% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala index 5967d66b13b5..bb4d66897565 100644 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/MergeIntoCommand.scala @@ -14,63 +14,66 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.delta.commands +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.sql.delta._ +import org.apache.spark.sql.delta.actions.{AddCDCFile, AddFile, FileAction} +import org.apache.spark.sql.delta.commands.merge.MergeIntoMaterializeSource +import org.apache.spark.sql.delta.files._ +import org.apache.spark.sql.delta.schema.{ImplicitMetadataOperation, SchemaUtils} +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.util.{AnalysisHelper, SetAccumulator} +import com.fasterxml.jackson.databind.annotation.JsonDeserialize + import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, BasePredicate, Expression, Literal, NamedExpression, PredicateHelper, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, BasePredicate, Expression, Literal, NamedExpression, PredicateHelper, UnsafeProjection} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap -import org.apache.spark.sql.delta._ -import org.apache.spark.sql.delta.actions.{AddCDCFile, AddFile, FileAction} -import org.apache.spark.sql.delta.commands.merge.MergeIntoMaterializeSource -import org.apache.spark.sql.delta.files._ -import org.apache.spark.sql.delta.schema.{ImplicitMetadataOperation, SchemaUtils} -import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.delta.util.{AnalysisHelper, SetAccumulator} -import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.LeafRunnableCommand import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataTypes, LongType, StructType} -import com.fasterxml.jackson.databind.annotation.JsonDeserialize - -import java.util.concurrent.TimeUnit - -import scala.collection.JavaConverters._ -import scala.collection.mutable - /** * Gluten overwrite Delta: * - * This file is copied from Delta 2.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 2.3.0. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). */ case class MergeDataSizes( - @JsonDeserialize(contentAs = classOf[java.lang.Long]) - rows: Option[Long] = None, - @JsonDeserialize(contentAs = classOf[java.lang.Long]) - files: Option[Long] = None, - @JsonDeserialize(contentAs = classOf[java.lang.Long]) - bytes: Option[Long] = None, - @JsonDeserialize(contentAs = classOf[java.lang.Long]) - partitions: Option[Long] = None) + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + rows: Option[Long] = None, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + files: Option[Long] = None, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + bytes: Option[Long] = None, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + partitions: Option[Long] = None) /** * Represents the state of a single merge clause: - * - merge clause's (optional) predicate - * - action type (insert, update, delete) - * - action's expressions + * - merge clause's (optional) predicate + * - action type (insert, update, delete) + * - action's expressions */ -case class MergeClauseStats(condition: Option[String], actionType: String, actionExpr: Seq[String]) +case class MergeClauseStats( + condition: Option[String], + actionType: String, + actionExpr: Seq[String]) object MergeClauseStats { def apply(mergeClause: DeltaMergeIntoClause): MergeClauseStats = { @@ -93,9 +96,10 @@ case class MergeStats( insertExprs: Seq[String], deleteConditionExpr: String, - // Newer expressions used in MERGE with any number of MATCHED/NOT MATCHED + // Newer expressions used in MERGE with any number of MATCHED/NOT MATCHED/NOT MATCHED BY SOURCE matchedStats: Seq[MergeClauseStats], notMatchedStats: Seq[MergeClauseStats], + notMatchedBySourceStats: Seq[MergeClauseStats], // Timings executionTimeMs: Long, @@ -126,8 +130,12 @@ case class MergeStats( targetPartitionsAddedTo: Option[Long], targetRowsCopied: Long, targetRowsUpdated: Long, + targetRowsMatchedUpdated: Long, + targetRowsNotMatchedBySourceUpdated: Long, targetRowsInserted: Long, targetRowsDeleted: Long, + targetRowsMatchedDeleted: Long, + targetRowsNotMatchedBySourceDeleted: Long, // MergeMaterializeSource stats materializeSourceReason: Option[String] = None, @@ -142,6 +150,7 @@ object MergeStats { condition: Expression, matchedClauses: Seq[DeltaMergeIntoMatchedClause], notMatchedClauses: Seq[DeltaMergeIntoNotMatchedClause], + notMatchedBySourceClauses: Seq[DeltaMergeIntoNotMatchedBySourceClause], isPartitioned: Boolean): MergeStats = { def metricValueIfPartitioned(metricName: String): Option[Long] = { @@ -152,9 +161,11 @@ object MergeStats { // Merge condition expression conditionExpr = condition.sql, - // Newer expressions used in MERGE with any number of MATCHED/NOT MATCHED + // Newer expressions used in MERGE with any number of MATCHED/NOT MATCHED/ + // NOT MATCHED BY SOURCE matchedStats = matchedClauses.map(MergeClauseStats(_)), notMatchedStats = notMatchedClauses.map(MergeClauseStats(_)), + notMatchedBySourceStats = notMatchedBySourceClauses.map(MergeClauseStats(_)), // Timings executionTimeMs = metrics("executionTimeMs").value, @@ -163,15 +174,17 @@ object MergeStats { // Data sizes of source and target at different stages of processing source = MergeDataSizes(rows = Some(metrics("numSourceRows").value)), - targetBeforeSkipping = MergeDataSizes( - files = Some(metrics("numTargetFilesBeforeSkipping").value), - bytes = Some(metrics("numTargetBytesBeforeSkipping").value)), - targetAfterSkipping = MergeDataSizes( - files = Some(metrics("numTargetFilesAfterSkipping").value), - bytes = Some(metrics("numTargetBytesAfterSkipping").value), - partitions = metricValueIfPartitioned("numTargetPartitionsAfterSkipping") - ), - sourceRowsInSecondScan = metrics.get("numSourceRowsInSecondScan").map(_.value).filter(_ >= 0), + targetBeforeSkipping = + MergeDataSizes( + files = Some(metrics("numTargetFilesBeforeSkipping").value), + bytes = Some(metrics("numTargetBytesBeforeSkipping").value)), + targetAfterSkipping = + MergeDataSizes( + files = Some(metrics("numTargetFilesAfterSkipping").value), + bytes = Some(metrics("numTargetBytesAfterSkipping").value), + partitions = metricValueIfPartitioned("numTargetPartitionsAfterSkipping")), + sourceRowsInSecondScan = + metrics.get("numSourceRowsInSecondScan").map(_.value).filter(_ >= 0), // Data change sizes targetFilesAdded = metrics("numTargetFilesAdded").value, @@ -184,49 +197,48 @@ object MergeStats { targetPartitionsAddedTo = metricValueIfPartitioned("numTargetPartitionsAddedTo"), targetRowsCopied = metrics("numTargetRowsCopied").value, targetRowsUpdated = metrics("numTargetRowsUpdated").value, + targetRowsMatchedUpdated = metrics("numTargetRowsMatchedUpdated").value, + targetRowsNotMatchedBySourceUpdated = metrics("numTargetRowsNotMatchedBySourceUpdated").value, targetRowsInserted = metrics("numTargetRowsInserted").value, targetRowsDeleted = metrics("numTargetRowsDeleted").value, + targetRowsMatchedDeleted = metrics("numTargetRowsMatchedDeleted").value, + targetRowsNotMatchedBySourceDeleted = metrics("numTargetRowsNotMatchedBySourceDeleted").value, // Deprecated fields updateConditionExpr = null, updateExprs = null, insertConditionExpr = null, insertExprs = null, - deleteConditionExpr = null - ) + deleteConditionExpr = null) } } /** * Performs a merge of a source query/table into a Delta table. * - * Issues an error message when the ON search_condition of the MERGE statement can match a single - * row from the target table with multiple rows of the source table-reference. + * Issues an error message when the ON search_condition of the MERGE statement can match + * a single row from the target table with multiple rows of the source table-reference. * * Algorithm: * - * Phase 1: Find the input files in target that are touched by the rows that satisfy the condition - * and verify that no two source rows match with the same target row. This is implemented as an - * inner-join using the given condition. See [[findTouchedFiles]] for more details. + * Phase 1: Find the input files in target that are touched by the rows that satisfy + * the condition and verify that no two source rows match with the same target row. + * This is implemented as an inner-join using the given condition. See [[findTouchedFiles]] + * for more details. * * Phase 2: Read the touched files again and write new files with updated and/or inserted rows. * * Phase 3: Use the Delta protocol to atomically remove the touched files and add the new files. * - * @param source - * Source data to merge from - * @param target - * Target table to merge into - * @param targetFileIndex - * TahoeFileIndex of the target table - * @param condition - * Condition for a source row to match with a target row - * @param matchedClauses - * All info related to matched clauses. - * @param notMatchedClauses - * All info related to not matched clause. - * @param migratedSchema - * The final schema of the target - may be changed by schema evolution. + * @param source Source data to merge from + * @param target Target table to merge into + * @param targetFileIndex TahoeFileIndex of the target table + * @param condition Condition for a source row to match with a target row + * @param matchedClauses All info related to matched clauses. + * @param notMatchedClauses All info related to not matched clauses. + * @param notMatchedBySourceClauses All info related to not matched by source clauses. + * @param migratedSchema The final schema of the target - may be changed by schema + * evolution. */ case class MergeIntoCommand( @transient source: LogicalPlan, @@ -235,18 +247,18 @@ case class MergeIntoCommand( condition: Expression, matchedClauses: Seq[DeltaMergeIntoMatchedClause], notMatchedClauses: Seq[DeltaMergeIntoNotMatchedClause], - migratedSchema: Option[StructType]) - extends LeafRunnableCommand + notMatchedBySourceClauses: Seq[DeltaMergeIntoNotMatchedBySourceClause], + migratedSchema: Option[StructType]) extends LeafRunnableCommand with DeltaCommand with PredicateHelper with AnalysisHelper with ImplicitMetadataOperation with MergeIntoMaterializeSource { - import org.apache.spark.sql.delta.commands.cdc.CDCReader._ - import MergeIntoCommand._ + import SQLMetrics._ + import org.apache.spark.sql.delta.commands.cdc.CDCReader._ override val canMergeSchema: Boolean = conf.getConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE) override val canOverwriteSchema: Boolean = false @@ -255,20 +267,18 @@ case class MergeIntoCommand( AttributeReference("num_affected_rows", LongType)(), AttributeReference("num_updated_rows", LongType)(), AttributeReference("num_deleted_rows", LongType)(), - AttributeReference("num_inserted_rows", LongType)() - ) + AttributeReference("num_inserted_rows", LongType)()) @transient private lazy val sc: SparkContext = SparkContext.getOrCreate() @transient private lazy val targetDeltaLog: DeltaLog = targetFileIndex.deltaLog - /** - * Map to get target output attributes by name. The case sensitivity of the map is set accordingly - * to Spark configuration. + * Map to get target output attributes by name. + * The case sensitivity of the map is set accordingly to Spark configuration. */ @transient private lazy val targetOutputAttributesMap: Map[String, Attribute] = { - val attrMap: Map[String, Attribute] = target.outputSet.view - .map(attr => attr.name -> attr) - .toMap + val attrMap: Map[String, Attribute] = target + .outputSet.view + .map(attr => attr.name -> attr).toMap if (conf.caseSensitiveAnalysis) { attrMap } else { @@ -277,10 +287,10 @@ case class MergeIntoCommand( } /** Whether this merge statement has only a single insert (NOT MATCHED) clause. */ - private def isSingleInsertOnly: Boolean = matchedClauses.isEmpty && notMatchedClauses.length == 1 - - /** Whether this merge statement has only MATCHED clauses. */ - private def isMatchedOnly: Boolean = notMatchedClauses.isEmpty && matchedClauses.nonEmpty + private def isSingleInsertOnly: Boolean = + matchedClauses.isEmpty && notMatchedBySourceClauses.isEmpty && notMatchedClauses.length == 1 + /** Whether this merge statement has no insert (NOT MATCHED) clause. */ + private def hasNoInserts: Boolean = notMatchedClauses.isEmpty // We over-count numTargetRowsDeleted when there are multiple matches; // this is the amount of the overcount, so we can subtract it to get a correct final metric. @@ -293,7 +303,15 @@ case class MergeIntoCommand( "numTargetRowsCopied" -> createMetric(sc, "number of target rows rewritten unmodified"), "numTargetRowsInserted" -> createMetric(sc, "number of inserted rows"), "numTargetRowsUpdated" -> createMetric(sc, "number of updated rows"), + "numTargetRowsMatchedUpdated" -> + createMetric(sc, "number of rows updated by a matched clause"), + "numTargetRowsNotMatchedBySourceUpdated" -> + createMetric(sc, "number of rows updated by a not matched by source clause"), "numTargetRowsDeleted" -> createMetric(sc, "number of deleted rows"), + "numTargetRowsMatchedDeleted" -> + createMetric(sc, "number of rows deleted by a matched clause"), + "numTargetRowsNotMatchedBySourceDeleted" -> + createMetric(sc, "number of rows deleted by a not matched by source clause"), "numTargetFilesBeforeSkipping" -> createMetric(sc, "number of target files before skipping"), "numTargetFilesAfterSkipping" -> createMetric(sc, "number of target files after skipping"), "numTargetFilesRemoved" -> createMetric(sc, "number of files removed to target"), @@ -317,8 +335,7 @@ case class MergeIntoCommand( "scanTimeMs" -> createTimingMetric(sc, "time taken to scan the files for matches"), "rewriteTimeMs" -> - createTimingMetric(sc, "time taken to rewrite the matched files") - ) + createTimingMetric(sc, "time taken to rewrite the matched files")) override def run(spark: SparkSession): Seq[Row] = { metrics("executionTimeMs").set(0) @@ -334,7 +351,7 @@ case class MergeIntoCommand( if (newNullColumn.isDefined) { throw new AnalysisException( s"""Cannot add column '${newNullColumn.get}' with type 'void'. Please explicitly specify a - |non-void type.""".stripMargin.replaceAll("\n", " ") + |non-void type.""".stripMargin.replaceAll("\n", " ") ) } } @@ -344,118 +361,107 @@ case class MergeIntoCommand( } else { // If it is determined that source should be materialized, wrap the execution with retries, // in case the data of the materialized source is lost. - runWithMaterializedSourceLostRetries(spark, targetFileIndex.deltaLog, metrics, runMerge) + runWithMaterializedSourceLostRetries( + spark, targetFileIndex.deltaLog, metrics, runMerge) } } protected def runMerge(spark: SparkSession): Seq[Row] = { recordDeltaOperation(targetDeltaLog, "delta.dml.merge") { val startTime = System.nanoTime() - targetDeltaLog.withNewTransaction { - deltaTxn => - if (target.schema.size != deltaTxn.metadata.schema.size) { - throw DeltaErrors.schemaChangedSinceAnalysis( - atAnalysis = target.schema, - latestSchema = deltaTxn.metadata.schema) - } + targetDeltaLog.withNewTransaction { deltaTxn => + if (hasBeenExecuted(deltaTxn, spark)) { + sendDriverMetrics(spark, metrics) + return Seq.empty + } + if (target.schema.size != deltaTxn.metadata.schema.size) { + throw DeltaErrors.schemaChangedSinceAnalysis( + atAnalysis = target.schema, latestSchema = deltaTxn.metadata.schema) + } - if (canMergeSchema) { - updateMetadata( - spark, - deltaTxn, - migratedSchema.getOrElse(target.schema), - deltaTxn.metadata.partitionColumns, - deltaTxn.metadata.configuration, - isOverwriteMode = false, - rearrangeOnly = false - ) - } + if (canMergeSchema) { + updateMetadata( + spark, deltaTxn, migratedSchema.getOrElse(target.schema), + deltaTxn.metadata.partitionColumns, deltaTxn.metadata.configuration, + isOverwriteMode = false, rearrangeOnly = false) + } - // If materialized, prepare the DF reading the materialize source - // Otherwise, prepare a regular DF from source plan. - val materializeSourceReason = prepareSourceDFAndReturnMaterializeReason( - spark, - source, - condition, - matchedClauses, - notMatchedClauses, - isSingleInsertOnly) - - val deltaActions = { - if (isSingleInsertOnly && spark.conf.get(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED)) { - writeInsertsOnlyWhenNoMatchedClauses(spark, deltaTxn) - } else { - val filesToRewrite = findTouchedFiles(spark, deltaTxn) - val newWrittenFiles = withStatusCode("DELTA", "Writing merged data") { - writeAllChanges(spark, deltaTxn, filesToRewrite) - } - filesToRewrite.map(_.remove) ++ newWrittenFiles + // If materialized, prepare the DF reading the materialize source + // Otherwise, prepare a regular DF from source plan. + val materializeSourceReason = prepareSourceDFAndReturnMaterializeReason( + spark, + source, + condition, + matchedClauses, + notMatchedClauses, + isSingleInsertOnly) + + val deltaActions = { + if (isSingleInsertOnly && spark.conf.get(DeltaSQLConf.MERGE_INSERT_ONLY_ENABLED)) { + writeInsertsOnlyWhenNoMatchedClauses(spark, deltaTxn) + } else { + val filesToRewrite = findTouchedFiles(spark, deltaTxn) + val newWrittenFiles = withStatusCode("DELTA", "Writing merged data") { + writeAllChanges(spark, deltaTxn, filesToRewrite) } + filesToRewrite.map(_.remove) ++ newWrittenFiles } + } - // Metrics should be recorded before commit (where they are written to delta logs). - metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) - deltaTxn.registerSQLMetrics(spark, metrics) - - // This is a best-effort sanity check. - if ( - metrics("numSourceRowsInSecondScan").value >= 0 && - metrics("numSourceRows").value != metrics("numSourceRowsInSecondScan").value - ) { - log.warn( - s"Merge source has ${metrics("numSourceRows")} rows in initial scan but " + - s"${metrics("numSourceRowsInSecondScan")} rows in second scan") - if (conf.getConf(DeltaSQLConf.MERGE_FAIL_IF_SOURCE_CHANGED)) { - throw DeltaErrors.sourceNotDeterministicInMergeException(spark) - } + val finalActions = createSetTransaction(spark, targetDeltaLog).toSeq ++ deltaActions + // Metrics should be recorded before commit (where they are written to delta logs). + metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) + deltaTxn.registerSQLMetrics(spark, metrics) + + // This is a best-effort sanity check. + if (metrics("numSourceRowsInSecondScan").value >= 0 && + metrics("numSourceRows").value != metrics("numSourceRowsInSecondScan").value) { + log.warn(s"Merge source has ${metrics("numSourceRows")} rows in initial scan but " + + s"${metrics("numSourceRowsInSecondScan")} rows in second scan") + if (conf.getConf(DeltaSQLConf.MERGE_FAIL_IF_SOURCE_CHANGED)) { + throw DeltaErrors.sourceNotDeterministicInMergeException(spark) } + } - deltaTxn.commit( - deltaActions, - DeltaOperations.Merge( - Option(condition.sql), - matchedClauses.map(DeltaOperations.MergePredicate(_)), - notMatchedClauses.map(DeltaOperations.MergePredicate(_))) - ) - - // Record metrics - var stats = MergeStats.fromMergeSQLMetrics( - metrics, - condition, - matchedClauses, - notMatchedClauses, - deltaTxn.metadata.partitionColumns.nonEmpty) - stats = stats.copy( - materializeSourceReason = Some(materializeSourceReason.toString), - materializeSourceAttempts = Some(attempt)) - - recordDeltaEvent(targetFileIndex.deltaLog, "delta.dml.merge.stats", data = stats) + deltaTxn.commitIfNeeded( + finalActions, + DeltaOperations.Merge( + Option(condition.sql), + matchedClauses.map(DeltaOperations.MergePredicate(_)), + notMatchedClauses.map(DeltaOperations.MergePredicate(_)), + notMatchedBySourceClauses.map(DeltaOperations.MergePredicate(_)))) + + // Record metrics + var stats = MergeStats.fromMergeSQLMetrics( + metrics, + condition, + matchedClauses, + notMatchedClauses, + notMatchedBySourceClauses, + deltaTxn.metadata.partitionColumns.nonEmpty) + stats = stats.copy( + materializeSourceReason = Some(materializeSourceReason.toString), + materializeSourceAttempts = Some(attempt)) + + recordDeltaEvent(targetFileIndex.deltaLog, "delta.dml.merge.stats", data = stats) } spark.sharedState.cacheManager.recacheByPlan(spark, target) } - // This is needed to make the SQL metrics visible in the Spark UI. Also this needs - // to be outside the recordMergeOperation because this method will update some metric. - val executionId = spark.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(spark.sparkContext, executionId, metrics.values.toSeq) - Seq( - Row( - metrics("numTargetRowsUpdated").value + metrics("numTargetRowsDeleted").value + - metrics("numTargetRowsInserted").value, - metrics("numTargetRowsUpdated").value, - metrics("numTargetRowsDeleted").value, - metrics("numTargetRowsInserted").value - )) + sendDriverMetrics(spark, metrics) + Seq(Row(metrics("numTargetRowsUpdated").value + metrics("numTargetRowsDeleted").value + + metrics("numTargetRowsInserted").value, metrics("numTargetRowsUpdated").value, + metrics("numTargetRowsDeleted").value, metrics("numTargetRowsInserted").value)) } /** * Find the target table files that contain the rows that satisfy the merge condition. This is - * implemented as an inner-join between the source query/table and the target table using the - * merge condition. + * implemented as an inner-join between the source query/table and the target table using + * the merge condition. */ private def findTouchedFiles( - spark: SparkSession, - deltaTxn: OptimisticTransaction + spark: SparkSession, + deltaTxn: OptimisticTransaction ): Seq[AddFile] = recordMergeOperation(sqlMetricName = "scanTimeMs") { // Accumulator to collect all the distinct touched files @@ -463,34 +469,40 @@ case class MergeIntoCommand( spark.sparkContext.register(touchedFilesAccum, TOUCHED_FILES_ACCUM_NAME) // UDFs to records touched files names and add them to the accumulator - val recordTouchedFileName = DeltaUDF - .intFromString { - fileName => - fileName.split(",").foreach(name => touchedFilesAccum.add(name)) - 1 + val recordTouchedFileName = DeltaUDF.intFromString { fileName => + // --- modified start + fileName.split(",").foreach(name => touchedFilesAccum.add(name)) + // --- modified end + 1 + }.asNondeterministic() + + // Prune non-matching files if we don't need to collect them for NOT MATCHED BY SOURCE clauses. + val dataSkippedFiles = + if (notMatchedBySourceClauses.isEmpty) { + val targetOnlyPredicates = + splitConjunctivePredicates(condition).filter(_.references.subsetOf(target.outputSet)) + deltaTxn.filterFiles(targetOnlyPredicates) + } else { + deltaTxn.filterFiles() } - .asNondeterministic() - - // Skip data based on the merge condition - val targetOnlyPredicates = - splitConjunctivePredicates(condition).filter(_.references.subsetOf(target.outputSet)) - val dataSkippedFiles = deltaTxn.filterFiles(targetOnlyPredicates) // UDF to increment metrics val incrSourceRowCountExpr = makeMetricUpdateUDF("numSourceRows") val sourceDF = getSourceDF() .filter(new Column(incrSourceRowCountExpr)) - // Apply inner join to between source and target using the merge condition to find matches + // Join the source and target table using the merge condition to find touched files. An inner + // join collects all candidate files for MATCHED clauses, a right outer join also includes + // candidates for NOT MATCHED BY SOURCE clauses. // In addition, we attach two columns // - a monotonically increasing row id for target rows to later identify whether the same // target row is modified by multiple user or not // - the target file name the row is from to later identify the files touched by matched rows - val targetDF = Dataset - .ofRows(spark, buildTargetPlanWithFiles(deltaTxn, dataSkippedFiles)) + val joinType = if (notMatchedBySourceClauses.isEmpty) "inner" else "right_outer" + val targetDF = buildTargetPlanWithFiles(spark, deltaTxn, dataSkippedFiles) .withColumn(ROW_ID_COL, monotonically_increasing_id()) .withColumn(FILE_NAME_COL, input_file_name()) - val joinToFindTouchedFiles = sourceDF.join(targetDF, new Column(condition), "inner") + val joinToFindTouchedFiles = sourceDF.join(targetDF, new Column(condition), joinType) // Process the matches from the inner join to record touched files and find multiple matches val collectTouchedFiles = joinToFindTouchedFiles @@ -542,16 +554,14 @@ case class MergeIntoCommand( logTrace(s"findTouchedFiles: matched files:\n\t${touchedFileNames.mkString("\n\t")}") val nameToAddFileMap = generateCandidateFileMap(targetDeltaLog.dataPath, dataSkippedFiles) - val touchedAddFiles = - touchedFileNames.map(f => getTouchedFile(targetDeltaLog.dataPath, f, nameToAddFileMap)) + val touchedAddFiles = touchedFileNames.map(f => + getTouchedFile(targetDeltaLog.dataPath, f, nameToAddFileMap)) // When the target table is empty, and the optimizer optimized away the join entirely // numSourceRows will be incorrectly 0. We need to scan the source table once to get the correct // metric here. - if ( - metrics("numSourceRows").value == 0 && - (dataSkippedFiles.isEmpty || targetDF.take(1).isEmpty) - ) { + if (metrics("numSourceRows").value == 0 && + (dataSkippedFiles.isEmpty || targetDF.take(1).isEmpty)) { val numSourceRows = sourceDF.count() metrics("numSourceRows").set(numSourceRows) } @@ -572,15 +582,15 @@ case class MergeIntoCommand( } /** - * This is an optimization of the case when there is no update clause for the merge. We perform an - * left anti join on the source data to find the rows to be inserted. + * This is an optimization of the case when there is no update clause for the merge. + * We perform an left anti join on the source data to find the rows to be inserted. * * This will currently only optimize for the case when there is a _single_ notMatchedClause. */ private def writeInsertsOnlyWhenNoMatchedClauses( spark: SparkSession, deltaTxn: OptimisticTransaction - ): Seq[FileAction] = recordMergeOperation(sqlMetricName = "rewriteTimeMs") { + ): Seq[FileAction] = recordMergeOperation(sqlMetricName = "rewriteTimeMs") { // UDFs to update metrics val incrSourceRowCountExpr = makeMetricUpdateUDF("numSourceRows") @@ -589,9 +599,8 @@ case class MergeIntoCommand( val outputColNames = getTargetOutputCols(deltaTxn).map(_.name) // we use head here since we know there is only a single notMatchedClause val outputExprs = notMatchedClauses.head.resolvedActions.map(_.expr) - val outputCols = outputExprs.zip(outputColNames).map { - case (expr, name) => - new Column(Alias(expr, name)()) + val outputCols = outputExprs.zip(outputColNames).map { case (expr, name) => + new Column(Alias(expr, name)()) } // source DataFrame @@ -606,10 +615,9 @@ case class MergeIntoCommand( val dataSkippedFiles = deltaTxn.filterFiles(targetOnlyPredicates) // target DataFrame - val targetDF = Dataset.ofRows(spark, buildTargetPlanWithFiles(deltaTxn, dataSkippedFiles)) + val targetDF = buildTargetPlanWithFiles(spark, deltaTxn, dataSkippedFiles) - val insertDf = sourceDF - .join(targetDF, new Column(condition), "leftanti") + val insertDf = sourceDF.join(targetDF, new Column(condition), "leftanti") .select(outputCols: _*) .filter(new Column(incrInsertedCountExpr)) @@ -652,11 +660,11 @@ case class MergeIntoCommand( * CDC_TYPE_COL_NAME used for handling CDC when enabled. */ private def writeAllChanges( - spark: SparkSession, - deltaTxn: OptimisticTransaction, - filesToRewrite: Seq[AddFile] + spark: SparkSession, + deltaTxn: OptimisticTransaction, + filesToRewrite: Seq[AddFile] ): Seq[FileAction] = recordMergeOperation(sqlMetricName = "rewriteTimeMs") { - import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} + import org.apache.spark.sql.catalyst.expressions.Literal.{TrueLiteral, FalseLiteral} val cdcEnabled = DeltaConfigs.CHANGE_DATA_FEED.fromMetaData(deltaTxn.metadata) @@ -685,32 +693,35 @@ case class MergeIntoCommand( // need to drop the duplicate matches. val isDeleteWithDuplicateMatchesAndCdc = multipleMatchDeleteOnlyOvercount.nonEmpty && cdcEnabled - // Generate a new logical plan that has same output attributes exprIds as the target plan. + // Generate a new target dataframe that has same output attributes exprIds as the target plan. // This allows us to apply the existing resolved update/insert expressions. - val newTarget = buildTargetPlanWithFiles(deltaTxn, filesToRewrite) - val joinType = - if ( - isMatchedOnly && - spark.conf.get(DeltaSQLConf.MERGE_MATCHED_ONLY_ENABLED) - ) { - "rightOuter" - } else { - "fullOuter" - } + val baseTargetDF = buildTargetPlanWithFiles(spark, deltaTxn, filesToRewrite) + val joinType = if (hasNoInserts && + spark.conf.get(DeltaSQLConf.MERGE_MATCHED_ONLY_ENABLED)) { + "rightOuter" + } else { + "fullOuter" + } logDebug(s"""writeAllChanges using $joinType join: | source.output: ${source.outputSet} | target.output: ${target.outputSet} | condition: $condition - | newTarget.output: ${newTarget.outputSet} + | newTarget.output: ${baseTargetDF.queryExecution.logical.outputSet} """.stripMargin) // UDFs to update metrics val incrSourceRowCountExpr = makeMetricUpdateUDF("numSourceRowsInSecondScan") val incrUpdatedCountExpr = makeMetricUpdateUDF("numTargetRowsUpdated") + val incrUpdatedMatchedCountExpr = makeMetricUpdateUDF("numTargetRowsMatchedUpdated") + val incrUpdatedNotMatchedBySourceCountExpr = + makeMetricUpdateUDF("numTargetRowsNotMatchedBySourceUpdated") val incrInsertedCountExpr = makeMetricUpdateUDF("numTargetRowsInserted") val incrNoopCountExpr = makeMetricUpdateUDF("numTargetRowsCopied") val incrDeletedCountExpr = makeMetricUpdateUDF("numTargetRowsDeleted") + val incrDeletedMatchedCountExpr = makeMetricUpdateUDF("numTargetRowsMatchedDeleted") + val incrDeletedNotMatchedBySourceCountExpr = + makeMetricUpdateUDF("numTargetRowsNotMatchedBySourceDeleted") // Apply an outer join to find both, matches and non-matches. We are adding two boolean fields // with value `true`, one to each side of the join. Whether this field is null or not after @@ -721,8 +732,7 @@ case class MergeIntoCommand( // insert clause. See above at isDeleteWithDuplicateMatchesAndCdc definition for more details. var sourceDF = getSourceDF() .withColumn(SOURCE_ROW_PRESENT_COL, new Column(incrSourceRowCountExpr)) - var targetDF = Dataset - .ofRows(spark, newTarget) + var targetDF = baseTargetDF .withColumn(TARGET_ROW_PRESENT_COL, lit(true)) if (isDeleteWithDuplicateMatchesAndCdc) { targetDF = targetDF.withColumn(TARGET_ROW_ID_COL, monotonically_increasing_id()) @@ -783,12 +793,13 @@ case class MergeIntoCommand( .add(CDC_TYPE_COLUMN_NAME, DataTypes.StringType) } - def updateOutput(resolvedActions: Seq[DeltaMergeAction]): Seq[Seq[Expression]] = { + def updateOutput(resolvedActions: Seq[DeltaMergeAction], incrMetricExpr: Expression) + : Seq[Seq[Expression]] = { val updateExprs = { // Generate update expressions and set ROW_DELETED_COL = false and // CDC_TYPE_COLUMN_NAME = CDC_TYPE_NOT_CDC val mainDataOutput = resolvedActions.map(_.expr) :+ FalseLiteral :+ - incrUpdatedCountExpr :+ CDC_TYPE_NOT_CDC + incrMetricExpr :+ CDC_TYPE_NOT_CDC if (cdcEnabled) { // For update preimage, we have do a no-op copy with ROW_DELETED_COL = false and // CDC_TYPE_COLUMN_NAME = CDC_TYPE_UPDATE_PREIMAGE and INCR_ROW_COUNT_COL as a no-op @@ -808,11 +819,11 @@ case class MergeIntoCommand( updateExprs.map(resolveOnJoinedPlan) } - def deleteOutput(): Seq[Seq[Expression]] = { + def deleteOutput(incrMetricExpr: Expression): Seq[Seq[Expression]] = { val deleteExprs = { // Generate expressions to set the ROW_DELETED_COL = true and CDC_TYPE_COLUMN_NAME = // CDC_TYPE_NOT_CDC - val mainDataOutput = targetOutputCols :+ TrueLiteral :+ incrDeletedCountExpr :+ + val mainDataOutput = targetOutputCols :+ TrueLiteral :+ incrMetricExpr :+ CDC_TYPE_NOT_CDC if (cdcEnabled) { // For delete we do a no-op copy with ROW_DELETED_COL = false, INCR_ROW_COUNT_COL as a @@ -827,7 +838,8 @@ case class MergeIntoCommand( deleteExprs.map(resolveOnJoinedPlan) } - def insertOutput(resolvedActions: Seq[DeltaMergeAction]): Seq[Seq[Expression]] = { + def insertOutput(resolvedActions: Seq[DeltaMergeAction], incrMetricExpr: Expression) + : Seq[Seq[Expression]] = { // Generate insert expressions and set ROW_DELETED_COL = false and // CDC_TYPE_COLUMN_NAME = CDC_TYPE_NOT_CDC val insertExprs = resolvedActions.map(_.expr) @@ -839,9 +851,9 @@ case class MergeIntoCommand( // isDeleteWithDuplicateMatchesAndCdc definition for more details. insertExprs :+ Alias(Literal(null), TARGET_ROW_ID_COL)() :+ UnresolvedAttribute(SOURCE_ROW_ID_COL) :+ - FalseLiteral :+ incrInsertedCountExpr :+ CDC_TYPE_NOT_CDC + FalseLiteral :+ incrMetricExpr :+ CDC_TYPE_NOT_CDC } else { - insertExprs :+ FalseLiteral :+ incrInsertedCountExpr :+ CDC_TYPE_NOT_CDC + insertExprs :+ FalseLiteral :+ incrMetricExpr :+ CDC_TYPE_NOT_CDC } ) if (cdcEnabled) { @@ -856,9 +868,18 @@ case class MergeIntoCommand( } def clauseOutput(clause: DeltaMergeIntoClause): Seq[Seq[Expression]] = clause match { - case u: DeltaMergeIntoMatchedUpdateClause => updateOutput(u.resolvedActions) - case _: DeltaMergeIntoMatchedDeleteClause => deleteOutput() - case i: DeltaMergeIntoNotMatchedInsertClause => insertOutput(i.resolvedActions) + case u: DeltaMergeIntoMatchedUpdateClause => + updateOutput(u.resolvedActions, And(incrUpdatedCountExpr, incrUpdatedMatchedCountExpr)) + case _: DeltaMergeIntoMatchedDeleteClause => + deleteOutput(And(incrDeletedCountExpr, incrDeletedMatchedCountExpr)) + case i: DeltaMergeIntoNotMatchedInsertClause => + insertOutput(i.resolvedActions, incrInsertedCountExpr) + case u: DeltaMergeIntoNotMatchedBySourceUpdateClause => + updateOutput( + u.resolvedActions, + And(incrUpdatedCountExpr, incrUpdatedNotMatchedBySourceCountExpr)) + case _: DeltaMergeIntoNotMatchedBySourceDeleteClause => + deleteOutput(And(incrDeletedCountExpr, incrDeletedNotMatchedBySourceCountExpr)) } def clauseCondition(clause: DeltaMergeIntoClause): Expression = { @@ -877,15 +898,16 @@ case class MergeIntoCommand( matchedOutputs = matchedClauses.map(clauseOutput), notMatchedConditions = notMatchedClauses.map(clauseCondition), notMatchedOutputs = notMatchedClauses.map(clauseOutput), - noopCopyOutput = resolveOnJoinedPlan( - targetOutputCols :+ FalseLiteral :+ incrNoopCountExpr :+ + notMatchedBySourceConditions = notMatchedBySourceClauses.map(clauseCondition), + notMatchedBySourceOutputs = notMatchedBySourceClauses.map(clauseOutput), + noopCopyOutput = + resolveOnJoinedPlan(targetOutputCols :+ FalseLiteral :+ incrNoopCountExpr :+ CDC_TYPE_NOT_CDC), deleteRowOutput = resolveOnJoinedPlan(targetOutputCols :+ TrueLiteral :+ TrueLiteral :+ CDC_TYPE_NOT_CDC), joinedAttributes = joinedPlan.output, joinedRowEncoder = joinedRowEncoder, - outputRowEncoder = outputRowEncoder - ) + outputRowEncoder = outputRowEncoder) var outputDF = Dataset.ofRows(spark, joinedPlan).mapPartitions(processor.processPartition)(outputRowEncoder) @@ -922,7 +944,7 @@ case class MergeIntoCommand( val (addedBytes, addedPartitions) = totalBytesAndDistinctPartitionValues(newFiles) metrics("numTargetFilesAdded") += newFiles.count(_.isInstanceOf[AddFile]) metrics("numTargetChangeFilesAdded") += newFiles.count(_.isInstanceOf[AddCDCFile]) - metrics("numTargetChangeFileBytes") += newFiles.collect { case f: AddCDCFile => f.size }.sum + metrics("numTargetChangeFileBytes") += newFiles.collect{ case f: AddCDCFile => f.size }.sum metrics("numTargetBytesAdded") += addedBytes metrics("numTargetPartitionsAddedTo") += addedPartitions if (multipleMatchDeleteOnlyOvercount.isDefined) { @@ -931,24 +953,29 @@ case class MergeIntoCommand( metrics("numTargetRowsDeleted").value - multipleMatchDeleteOnlyOvercount.get assert(actualRowsDeleted >= 0) metrics("numTargetRowsDeleted").set(actualRowsDeleted) + val actualRowsMatchedDeleted = + metrics("numTargetRowsMatchedDeleted").value - multipleMatchDeleteOnlyOvercount.get + assert(actualRowsMatchedDeleted >= 0) + metrics("numTargetRowsMatchedDeleted").set(actualRowsMatchedDeleted) } newFiles } + /** - * Build a new logical plan using the given `files` that has the same output columns (exprIds) as - * the `target` logical plan, so that existing update/insert expressions can be applied on this - * new plan. + * Build a new logical plan using the given `files` that has the same output columns (exprIds) + * as the `target` logical plan, so that existing update/insert expressions can be applied + * on this new plan. */ private def buildTargetPlanWithFiles( - deltaTxn: OptimisticTransaction, - files: Seq[AddFile]): LogicalPlan = { + spark: SparkSession, + deltaTxn: OptimisticTransaction, + files: Seq[AddFile]): DataFrame = { val targetOutputCols = getTargetOutputCols(deltaTxn) val targetOutputColsMap = { val colsMap: Map[String, NamedExpression] = targetOutputCols.view - .map(col => col.name -> col) - .toMap + .map(col => col.name -> col).toMap if (conf.caseSensitiveAnalysis) { colsMap } else { @@ -968,8 +995,7 @@ case class MergeIntoCommand( // We can ignore the new columns which aren't yet AttributeReferences. targetOutputCols.collect { case a: AttributeReference => a }, catalogTbl, - isStreaming - ) + isStreaming) } // In case of schema evolution & column mapping, we would also need to rebuild the file format @@ -987,14 +1013,11 @@ case class MergeIntoCommand( // create an alias val aliases = plan.output.map { case newAttrib: AttributeReference => - val existingTargetAttrib = targetOutputColsMap - .get(newAttrib.name) + val existingTargetAttrib = targetOutputColsMap.get(newAttrib.name) .getOrElse { throw DeltaErrors.failedFindAttributeInOutputColumns( - newAttrib.name, - targetOutputCols.mkString(",")) - } - .asInstanceOf[AttributeReference] + newAttrib.name, targetOutputCols.mkString(",")) + }.asInstanceOf[AttributeReference] if (existingTargetAttrib.exprId == newAttrib.exprId) { // It's not valid to alias an expression to its own exprId (this is considered a @@ -1005,7 +1028,7 @@ case class MergeIntoCommand( } } - Project(aliases, plan) + Dataset.ofRows(spark, Project(aliases, plan)) } /** Expressions to increment SQL metrics */ @@ -1016,18 +1039,20 @@ case class MergeIntoCommand( } private def getTargetOutputCols(txn: OptimisticTransaction): Seq[NamedExpression] = { - txn.metadata.schema.map { - col => - targetOutputAttributesMap - .get(col.name) - .map(a => AttributeReference(col.name, col.dataType, col.nullable)(a.exprId)) - .getOrElse(Alias(Literal(null), col.name)()) + txn.metadata.schema.map { col => + targetOutputAttributesMap + .get(col.name) + .map { a => + AttributeReference(col.name, col.dataType, col.nullable)(a.exprId) + } + .getOrElse(Alias(Literal(null), col.name)() + ) } } /** - * Repartitions the output DataFrame by the partition columns if table is partitioned and - * `merge.repartitionBeforeWrite.enabled` is set to true. + * Repartitions the output DataFrame by the partition columns if table is partitioned + * and `merge.repartitionBeforeWrite.enabled` is set to true. */ protected def repartitionIfNeeded( spark: SparkSession, @@ -1043,10 +1068,8 @@ case class MergeIntoCommand( /** * Execute the given `thunk` and return its result while recording the time taken to do it. * - * @param sqlMetricName - * name of SQL metric to update with the time taken by the thunk - * @param thunk - * the code to execute + * @param sqlMetricName name of SQL metric to update with the time taken by the thunk + * @param thunk the code to execute */ private def recordMergeOperation[A](sqlMetricName: String = null)(thunk: => A): A = { val startTimeNs = System.nanoTime() @@ -1060,7 +1083,6 @@ case class MergeIntoCommand( } object MergeIntoCommand { - /** * Spark UI will track all normal accumulators along with Spark tasks to show them on Web UI. * However, the accumulator used by `MergeIntoCommand` can store a very large value since it @@ -1081,31 +1103,29 @@ object MergeIntoCommand { val INCR_ROW_COUNT_COL = "_incr_row_count_" /** - * @param targetRowHasNoMatch - * whether a joined row is a target row with no match in the source table - * @param sourceRowHasNoMatch - * whether a joined row is a source row with no match in the target table - * @param matchedConditions - * condition for each match clause - * @param matchedOutputs - * corresponding output for each match clause. for each clause, we have 1-3 output rows, each of - * which is a sequence of expressions to apply to the joined row - * @param notMatchedConditions - * condition for each not-matched clause - * @param notMatchedOutputs - * corresponding output for each not-matched clause. for each clause, we have 1-2 output rows, - * each of which is a sequence of expressions to apply to the joined row - * @param noopCopyOutput - * no-op expression to copy a target row to the output - * @param deleteRowOutput - * expression to drop a row from the final output. this is used for source rows that don't match - * any not-matched clauses - * @param joinedAttributes - * schema of our outer-joined dataframe - * @param joinedRowEncoder - * joinedDF row encoder - * @param outputRowEncoder - * final output row encoder + * @param targetRowHasNoMatch whether a joined row is a target row with no match in the source + * table + * @param sourceRowHasNoMatch whether a joined row is a source row with no match in the target + * table + * @param matchedConditions condition for each match clause + * @param matchedOutputs corresponding output for each match clause. for each clause, we + * have 1-3 output rows, each of which is a sequence of expressions + * to apply to the joined row + * @param notMatchedConditions condition for each not-matched clause + * @param notMatchedOutputs corresponding output for each not-matched clause. for each clause, + * we have 1-2 output rows, each of which is a sequence of + * expressions to apply to the joined row + * @param notMatchedBySourceConditions condition for each not-matched-by-source clause + * @param notMatchedBySourceOutputs corresponding output for each not-matched-by-source + * clause. for each clause, we have 1-3 output rows, each of + * which is a sequence of expressions to apply to the joined + * row + * @param noopCopyOutput no-op expression to copy a target row to the output + * @param deleteRowOutput expression to drop a row from the final output. this is used for + * source rows that don't match any not-matched clauses + * @param joinedAttributes schema of our outer-joined dataframe + * @param joinedRowEncoder joinedDF row encoder + * @param outputRowEncoder final output row encoder */ class JoinedRowProcessor( targetRowHasNoMatch: Expression, @@ -1114,12 +1134,13 @@ object MergeIntoCommand { matchedOutputs: Seq[Seq[Seq[Expression]]], notMatchedConditions: Seq[Expression], notMatchedOutputs: Seq[Seq[Seq[Expression]]], + notMatchedBySourceConditions: Seq[Expression], + notMatchedBySourceOutputs: Seq[Seq[Seq[Expression]]], noopCopyOutput: Seq[Expression], deleteRowOutput: Seq[Expression], joinedAttributes: Seq[Attribute], joinedRowEncoder: ExpressionEncoder[Row], - outputRowEncoder: ExpressionEncoder[Row]) - extends Serializable { + outputRowEncoder: ExpressionEncoder[Row]) extends Serializable { private def generateProjection(exprs: Seq[Expression]): UnsafeProjection = { UnsafeProjection.create(exprs, joinedAttributes) @@ -1137,6 +1158,8 @@ object MergeIntoCommand { val matchedProjs = matchedOutputs.map(_.map(generateProjection)) val notMatchedPreds = notMatchedConditions.map(generatePredicate) val notMatchedProjs = notMatchedOutputs.map(_.map(generateProjection)) + val notMatchedBySourcePreds = notMatchedBySourceConditions.map(generatePredicate) + val notMatchedBySourceProjs = notMatchedBySourceOutputs.map(_.map(generateProjection)) val noopCopyProj = generateProjection(noopCopyOutput) val deleteRowProj = generateProjection(deleteRowOutput) val outputProj = UnsafeProjection.create(outputRowEncoder.schema) @@ -1145,35 +1168,33 @@ object MergeIntoCommand { // then CDC must be disabled and it's the column after our output cols def shouldDeleteRow(row: InternalRow): Boolean = { row.getBoolean( - outputRowEncoder.schema - .getFieldIndex(ROW_DROPPED_COL) + outputRowEncoder.schema.getFieldIndex(ROW_DROPPED_COL) .getOrElse(outputRowEncoder.schema.fields.size) ) } def processRow(inputRow: InternalRow): Iterator[InternalRow] = { - if (targetRowHasNoMatchPred.eval(inputRow)) { - // Target row did not match any source row, so just copy it to the output - Iterator(noopCopyProj.apply(inputRow)) + // Identify which set of clauses to execute: matched, not-matched or not-matched-by-source + val (predicates, projections, noopAction) = if (targetRowHasNoMatchPred.eval(inputRow)) { + // Target row did not match any source row, so update the target row. + (notMatchedBySourcePreds, notMatchedBySourceProjs, noopCopyProj) + } else if (sourceRowHasNoMatchPred.eval(inputRow)) { + // Source row did not match with any target row, so insert the new source row + (notMatchedPreds, notMatchedProjs, deleteRowProj) } else { - // identify which set of clauses to execute: matched or not-matched ones - val (predicates, projections, noopAction) = if (sourceRowHasNoMatchPred.eval(inputRow)) { - // Source row did not match with any target row, so insert the new source row - (notMatchedPreds, notMatchedProjs, deleteRowProj) - } else { - // Source row matched with target row, so update the target row - (matchedPreds, matchedProjs, noopCopyProj) - } + // Source row matched with target row, so update the target row + (matchedPreds, matchedProjs, noopCopyProj) + } - // find (predicate, projection) pair whose predicate satisfies inputRow - val pair = - (predicates.zip(projections)).find { case (predicate, _) => predicate.eval(inputRow) } + // find (predicate, projection) pair whose predicate satisfies inputRow + val pair = (predicates zip projections).find { + case (predicate, _) => predicate.eval(inputRow) + } - pair match { - case Some((_, projections)) => - projections.map(_.apply(inputRow)).iterator - case None => Iterator(noopAction.apply(inputRow)) - } + pair match { + case Some((_, projections)) => + projections.map(_.apply(inputRow)).iterator + case None => Iterator(noopAction.apply(inputRow)) } } @@ -1183,7 +1204,9 @@ object MergeIntoCommand { .map(toRow) .flatMap(processRow) .filter(!shouldDeleteRow(_)) - .map(notDeletedInternalRow => fromRow(outputProj(notDeletedInternalRow))) + .map { notDeletedInternalRow => + fromRow(outputProj(notDeletedInternalRow)) + } } } diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala similarity index 51% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala index e69b3aa65931..7fa2c97d9006 100644 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala @@ -14,24 +14,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.delta.commands -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext.SPARK_JOB_GROUP_ID -import org.apache.spark.sql.{AnalysisException, Encoders, Row, SparkSession} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} +import java.util.ConcurrentModificationException + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.delta.skipping.MultiDimClustering import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.DeltaOperations.Operation -import org.apache.spark.sql.delta.actions.{Action, AddFile, FileAction, RemoveFile} +import org.apache.spark.sql.delta.actions.{Action, AddFile, DeletionVectorDescriptor, FileAction, RemoveFile} import org.apache.spark.sql.delta.commands.OptimizeTableCommandOverwrites.{getDeltaLogClickhouse, groupFilesIntoBinsClickhouse, runOptimizeBinJobClickhouse} import org.apache.spark.sql.delta.commands.optimize._ import org.apache.spark.sql.delta.files.SQLMetricsReporting import org.apache.spark.sql.delta.schema.SchemaUtils -import org.apache.spark.sql.delta.skipping.MultiDimClustering import org.apache.spark.sql.delta.sources.DeltaSQLConf + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext.SPARK_JOB_GROUP_ID +import org.apache.spark.sql.{AnalysisException, Encoders, Row, SparkSession} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Expression} import org.apache.spark.sql.execution.command.{LeafRunnableCommand, RunnableCommand} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.execution.datasources.v2.clickhouse.metadata.AddMergeTreeParts import org.apache.spark.sql.execution.datasources.v2.clickhouse.utils.CHDataSourceUtils import org.apache.spark.sql.execution.metric.SQLMetric @@ -39,15 +46,13 @@ import org.apache.spark.sql.execution.metric.SQLMetrics.createMetric import org.apache.spark.sql.types._ import org.apache.spark.util.{SystemClock, ThreadUtils} -import java.util.ConcurrentModificationException - -import scala.collection.mutable.ArrayBuffer - /** * Gluten overwrite Delta: * - * This file is copied from Delta 2.2.0. It is modified in: - * 1. getDeltaLogClickhouse 2. runOptimizeBinJobClickhouse 3. groupFilesIntoBinsClickhouse + * This file is copied from Delta 2.3.0. It is modified in: + * 1. getDeltaLogClickhouse + * 2. runOptimizeBinJobClickhouse + * 3. groupFilesIntoBinsClickhouse */ /** Base class defining abstract optimize command */ @@ -59,16 +64,12 @@ abstract class OptimizeTableCommandBase extends RunnableCommand with DeltaComman /** * Validates ZOrderBy columns - * - validates that partitions columns are not used in `unresolvedZOrderByCols` - * - validates that we already collect stats for all the columns used in - * `unresolvedZOrderByCols` + * - validates that partitions columns are not used in `unresolvedZOrderByCols` + * - validates that we already collect stats for all the columns used in `unresolvedZOrderByCols` * - * @param spark - * [[SparkSession]] to use - * @param txn - * the [[OptimisticTransaction]] being used to optimize - * @param unresolvedZOrderByCols - * Seq of [[UnresolvedAttribute]] corresponding to zOrderBy columns + * @param spark [[SparkSession]] to use + * @param txn the [[OptimisticTransaction]] being used to optimize + * @param unresolvedZOrderByCols Seq of [[UnresolvedAttribute]] corresponding to zOrderBy columns */ def validateZorderByColumns( spark: SparkSession, @@ -80,32 +81,32 @@ abstract class OptimizeTableCommandBase extends RunnableCommand with DeltaComman val dataSchema = StructType(metadata.schema.filterNot(c => partitionColumns.contains(c.name))) val df = spark.createDataFrame(new java.util.ArrayList[Row](), dataSchema) - val checkColStat = - spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_ZORDER_COL_STAT_CHECK) + val checkColStat = spark.sessionState.conf.getConf( + DeltaSQLConf.DELTA_OPTIMIZE_ZORDER_COL_STAT_CHECK) val statCollectionSchema = txn.snapshot.statCollectionSchema val colsWithoutStats = ArrayBuffer[String]() - unresolvedZOrderByCols.foreach { - colAttribute => - val colName = colAttribute.name - if (checkColStat) { - try { - SchemaUtils.findColumnPosition(colAttribute.nameParts, statCollectionSchema) - } catch { - case e: AnalysisException if e.getMessage.contains("Couldn't find column") => - colsWithoutStats.append(colName) - } - } - val isNameEqual = spark.sessionState.conf.resolver - if (partitionColumns.find(isNameEqual(_, colName)).nonEmpty) { - throw DeltaErrors.zOrderingOnPartitionColumnException(colName) - } - if (df.queryExecution.analyzed.resolve(colAttribute.nameParts, isNameEqual).isEmpty) { - throw DeltaErrors.zOrderingColumnDoesNotExistException(colName) + unresolvedZOrderByCols.foreach { colAttribute => + val colName = colAttribute.name + if (checkColStat) { + try { + SchemaUtils.findColumnPosition(colAttribute.nameParts, statCollectionSchema) + } catch { + case e: AnalysisException if e.getMessage.contains("Couldn't find column") => + colsWithoutStats.append(colName) } + } + val isNameEqual = spark.sessionState.conf.resolver + if (partitionColumns.find(isNameEqual(_, colName)).nonEmpty) { + throw DeltaErrors.zOrderingOnPartitionColumnException(colName) + } + if (df.queryExecution.analyzed.resolve(colAttribute.nameParts, isNameEqual).isEmpty) { + throw DeltaErrors.zOrderingColumnDoesNotExistException(colName) + } } if (checkColStat && colsWithoutStats.nonEmpty) { - throw DeltaErrors.zOrderingOnColumnWithNoStatsException(colsWithoutStats.toSeq, spark) + throw DeltaErrors.zOrderingOnColumnWithNoStatsException( + colsWithoutStats.toSeq, spark) } } } @@ -121,15 +122,15 @@ case class OptimizeTableCommand( tableId: Option[TableIdentifier], userPartitionPredicates: Seq[String], options: Map[String, String])(val zOrderBy: Seq[UnresolvedAttribute]) - extends OptimizeTableCommandBase - with LeafRunnableCommand { + extends OptimizeTableCommandBase with LeafRunnableCommand { override val otherCopyArgs: Seq[AnyRef] = zOrderBy :: Nil override def run(sparkSession: SparkSession): Seq[Row] = { + // --- modified start CHDataSourceUtils.ensureClickHouseTableV2(tableId, sparkSession) - val deltaLog = getDeltaLogClickhouse(sparkSession, path, tableId, "OPTIMIZE", options) + // --- modified end val txn = deltaLog.startTransaction() if (txn.readVersion == -1) { @@ -140,10 +141,12 @@ case class OptimizeTableCommand( // Parse the predicate expression into Catalyst expression and verify only simple filters // on partition columns are present - val partitionPredicates = userPartitionPredicates.flatMap { - predicate => + val partitionPredicates = userPartitionPredicates.flatMap { predicate => val predicates = parsePredicates(sparkSession, predicate) - verifyPartitionPredicates(sparkSession, partitionColumns, predicates) + verifyPartitionPredicates( + sparkSession, + partitionColumns, + predicates) predicates } @@ -155,24 +158,19 @@ case class OptimizeTableCommand( } /** - * Optimize job which compacts small files into larger files to reduce the number of files and - * potentially allow more efficient reads. + * Optimize job which compacts small files into larger files to reduce + * the number of files and potentially allow more efficient reads. * - * @param sparkSession - * Spark environment reference. - * @param txn - * The transaction used to optimize this table - * @param partitionPredicate - * List of partition predicates to select subset of files to optimize. + * @param sparkSession Spark environment reference. + * @param txn The transaction used to optimize this table + * @param partitionPredicate List of partition predicates to select subset of files to optimize. */ class OptimizeExecutor( sparkSession: SparkSession, txn: OptimisticTransaction, partitionPredicate: Seq[Expression], zOrderByColumns: Seq[String]) - extends DeltaCommand - with SQLMetricsReporting - with Serializable { + extends DeltaCommand with SQLMetricsReporting with Serializable { /** Timestamp to use in [[FileAction]] */ private val operationTimestamp = new SystemClock().getTimeMillis() @@ -181,63 +179,72 @@ class OptimizeExecutor( def optimize(): Seq[Row] = { recordDeltaOperation(txn.deltaLog, "delta.optimize") { - val minFileSize = - sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE) - val maxFileSize = - sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE) + // --- modified start + val isMergeTreeFormat = ClickHouseConfig + .isMergeTreeFormatEngine(txn.deltaLog.unsafeVolatileMetadata.configuration) + // --- modified end + val minFileSize = sparkSession.sessionState.conf.getConf( + DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE) + val maxFileSize = sparkSession.sessionState.conf.getConf( + DeltaSQLConf.DELTA_OPTIMIZE_MAX_FILE_SIZE) require(minFileSize > 0, "minFileSize must be > 0") require(maxFileSize > 0, "maxFileSize must be > 0") - val candidateFiles = txn.filterFiles(partitionPredicate) + val candidateFiles = txn.filterFiles(partitionPredicate, keepNumRecords = true) val partitionSchema = txn.metadata.partitionSchema - // select all files in case of multi-dimensional clustering - val filesToProcess = candidateFiles.filter(_.size < minFileSize || isMultiDimClustering) - val partitionsToCompact = filesToProcess - .groupBy(file => (file.asInstanceOf[AddMergeTreeParts].bucketNum, file.partitionValues)) - .toSeq - - val jobs = groupFilesIntoBinsClickhouse(partitionsToCompact, maxFileSize) - + val maxDeletedRowsRatio = sparkSession.sessionState.conf.getConf( + DeltaSQLConf.DELTA_OPTIMIZE_MAX_DELETED_ROWS_RATIO) + val filesToProcess = pruneCandidateFileList(minFileSize, maxDeletedRowsRatio, candidateFiles) + // --- modified start val maxThreads = sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_MAX_THREADS) - val updates = ThreadUtils - .parmap(jobs, "OptimizeJob", maxThreads) { - partitionBinGroup => - runOptimizeBinJobClickhouse( - txn, - partitionBinGroup._1._2, - partitionBinGroup._1._1, - partitionBinGroup._2, - maxFileSize) - } - .flatten + val (updates, jobs) = if (isMergeTreeFormat) { + val partitionsToCompact = filesToProcess + .groupBy(file => (file.asInstanceOf[AddMergeTreeParts].bucketNum, file.partitionValues)) + .toSeq + val jobs = groupFilesIntoBinsClickhouse(partitionsToCompact, maxFileSize) + (ThreadUtils.parmap(jobs, "OptimizeJob", maxThreads) { partitionBinGroup => + // --- modified start + runOptimizeBinJobClickhouse( + txn, + partitionBinGroup._1._2, + partitionBinGroup._1._1, + partitionBinGroup._2, + maxFileSize) + // --- modified end + }.flatten, jobs) + } else { + val partitionsToCompact = filesToProcess.groupBy(_.partitionValues).toSeq + val jobs = groupFilesIntoBins(partitionsToCompact, maxFileSize) + (ThreadUtils.parmap(jobs, "OptimizeJob", maxThreads) { partitionBinGroup => + runOptimizeBinJob(txn, partitionBinGroup._1, partitionBinGroup._2, maxFileSize) + }.flatten, jobs) + } + // --- modified end val addedFiles = updates.collect { case a: AddFile => a } val removedFiles = updates.collect { case r: RemoveFile => r } + val removedDVs = filesToProcess.filter(_.deletionVector != null).map(_.deletionVector).toSeq if (addedFiles.size > 0) { val operation = DeltaOperations.Optimize(partitionPredicate.map(_.sql), zOrderByColumns) - val metrics = createMetrics(sparkSession.sparkContext, addedFiles, removedFiles) - commitAndRetry(txn, operation, updates, metrics) { - newTxn => - val newPartitionSchema = newTxn.metadata.partitionSchema - val candidateSetOld = candidateFiles.map(_.path).toSet - val candidateSetNew = newTxn.filterFiles(partitionPredicate).map(_.path).toSet - - // As long as all of the files that we compacted are still part of the table, - // and the partitioning has not changed it is valid to continue to try - // and commit this checkpoint. - if ( - candidateSetOld.subsetOf(candidateSetNew) && partitionSchema == newPartitionSchema - ) { - true - } else { - val deleted = candidateSetOld -- candidateSetNew - logWarning( - s"The following compacted files were delete " + - s"during checkpoint ${deleted.mkString(",")}. Aborting the compaction.") - false - } + val metrics = createMetrics(sparkSession.sparkContext, addedFiles, removedFiles, removedDVs) + commitAndRetry(txn, operation, updates, metrics) { newTxn => + val newPartitionSchema = newTxn.metadata.partitionSchema + val candidateSetOld = candidateFiles.map(_.path).toSet + val candidateSetNew = newTxn.filterFiles(partitionPredicate).map(_.path).toSet + + // As long as all of the files that we compacted are still part of the table, + // and the partitioning has not changed it is valid to continue to try + // and commit this checkpoint. + if (candidateSetOld.subsetOf(candidateSetNew) && partitionSchema == newPartitionSchema) { + true + } else { + val deleted = candidateSetOld -- candidateSetNew + logWarning(s"The following compacted files were delete " + + s"during checkpoint ${deleted.mkString(",")}. Aborting the compaction.") + false + } } } @@ -249,37 +256,65 @@ class OptimizeExecutor( optimizeStats.totalConsideredFiles = candidateFiles.size optimizeStats.totalFilesSkipped = optimizeStats.totalConsideredFiles - removedFiles.size optimizeStats.totalClusterParallelism = sparkSession.sparkContext.defaultParallelism + val numTableColumns = txn.snapshot.metadata.schema.size + optimizeStats.numTableColumns = numTableColumns + optimizeStats.numTableColumnsWithStats = + DeltaConfigs.DATA_SKIPPING_NUM_INDEXED_COLS.fromMetaData(txn.snapshot.metadata) + .min(numTableColumns) + if (removedDVs.size > 0) { + optimizeStats.deletionVectorStats = Some(DeletionVectorStats( + numDeletionVectorsRemoved = removedDVs.size, + numDeletionVectorRowsRemoved = removedDVs.map(_.cardinality).sum)) + } if (isMultiDimClustering) { val inputFileStats = ZOrderFileStats(removedFiles.size, removedFiles.map(_.size.getOrElse(0L)).sum) - optimizeStats.zOrderStats = Some( - ZOrderStats( - strategyName = "all", // means process all files in a partition - inputCubeFiles = ZOrderFileStats(0, 0), - inputOtherFiles = inputFileStats, - inputNumCubes = 0, - mergedFiles = inputFileStats, - // There will one z-cube for each partition - numOutputCubes = optimizeStats.numPartitionsOptimized - )) + optimizeStats.zOrderStats = Some(ZOrderStats( + strategyName = "all", // means process all files in a partition + inputCubeFiles = ZOrderFileStats(0, 0), + inputOtherFiles = inputFileStats, + inputNumCubes = 0, + mergedFiles = inputFileStats, + // There will one z-cube for each partition + numOutputCubes = optimizeStats.numPartitionsOptimized)) } return Seq(Row(txn.deltaLog.dataPath.toString, optimizeStats.toOptimizeMetrics)) } } + /** + * Helper method to prune the list of selected files based on fileSize and ratio of + * deleted rows according to the deletion vector in [[AddFile]]. + */ + private def pruneCandidateFileList( + minFileSize: Long, maxDeletedRowsRatio: Double, files: Seq[AddFile]): Seq[AddFile] = { + + // Select all files in case of multi-dimensional clustering + if (isMultiDimClustering) return files + + def shouldCompactBecauseOfDeletedRows(file: AddFile): Boolean = { + // Always compact files with DVs but without numRecords stats. + // This may be overly aggressive, but it fixes the problem in the long-term, + // as the compacted files will have stats. + (file.deletionVector != null && file.numPhysicalRecords.isEmpty) || + file.deletedToPhysicalRecordsRatio.getOrElse(0d) > maxDeletedRowsRatio + } + + // Select files that are small or have too many deleted rows + files.filter( + addFile => addFile.size < minFileSize || shouldCompactBecauseOfDeletedRows(addFile)) + } + /** * Utility methods to group files into bins for optimize. * - * @param partitionsToCompact - * List of files to compact group by partition. Partition is defined by the partition values - * (partCol -> partValue) - * @param maxTargetFileSize - * Max size (in bytes) of the compaction output file. - * @return - * Sequence of bins. Each bin contains one or more files from the same partition and targeted - * for one output file. + * @param partitionsToCompact List of files to compact group by partition. + * Partition is defined by the partition values (partCol -> partValue) + * @param maxTargetFileSize Max size (in bytes) of the compaction output file. + * @return Sequence of bins. Each bin contains one or more files from the same + * partition and targeted for one output file. */ private def groupFilesIntoBins( partitionsToCompact: Seq[(Map[String, String], Seq[AddFile])], @@ -291,47 +326,42 @@ class OptimizeExecutor( val currentBin = new ArrayBuffer[AddFile]() var currentBinSize = 0L - files.sortBy(_.size).foreach { - file => - // Generally, a bin is a group of existing files, whose total size does not exceed the - // desired maxFileSize. They will be coalesced into a single output file. - // However, if isMultiDimClustering = true, all files in a partition will be read by the - // same job, the data will be range-partitioned and - // umFiles = totalFileSize / maxFileSize will be produced. See below. - if (file.size + currentBinSize > maxTargetFileSize && !isMultiDimClustering) { - bins += currentBin.toVector - currentBin.clear() - currentBin += file - currentBinSize = file.size - } else { - currentBin += file - currentBinSize += file.size - } + files.sortBy(_.size).foreach { file => + // Generally, a bin is a group of existing files, whose total size does not exceed the + // desired maxFileSize. They will be coalesced into a single output file. + // However, if isMultiDimClustering = true, all files in a partition will be read by the + // same job, the data will be range-partitioned and numFiles = totalFileSize / maxFileSize + // will be produced. See below. + if (file.size + currentBinSize > maxTargetFileSize && !isMultiDimClustering) { + bins += currentBin.toVector + currentBin.clear() + currentBin += file + currentBinSize = file.size + } else { + currentBin += file + currentBinSize += file.size + } } if (currentBin.nonEmpty) { bins += currentBin.toVector } - bins - .map(b => (partition, b)) - // select bins that have at least two files or in case of multi-dim clustering - // select all bins - .filter(_._2.size > 1 || isMultiDimClustering) + bins.filter { bin => + bin.size > 1 || // bin has more than one file or + (bin.size == 1 && bin(0).deletionVector != null) || // single file in the bin has a DV or + isMultiDimClustering // multi-clustering + }.map(b => (partition, b)) } } /** * Utility method to run a Spark job to compact the files in given bin * - * @param txn - * [[OptimisticTransaction]] instance in use to commit the changes to DeltaLog. - * @param partition - * Partition values of the partition that files in [[bin]] belongs to. - * @param bin - * List of files to compact into one large file. - * @param maxFileSize - * Targeted output file size in bytes + * @param txn [[OptimisticTransaction]] instance in use to commit the changes to DeltaLog. + * @param partition Partition values of the partition that files in [[bin]] belongs to. + * @param bin List of files to compact into one large file. + * @param maxFileSize Targeted output file size in bytes */ private def runOptimizeBinJob( txn: OptimisticTransaction, @@ -344,10 +374,13 @@ class OptimizeExecutor( val repartitionDF = if (isMultiDimClustering) { val totalSize = bin.map(_.size).sum val approxNumFiles = Math.max(1, totalSize / maxFileSize).toInt - MultiDimClustering.cluster(input, approxNumFiles, zOrderByColumns) + MultiDimClustering.cluster( + input, + approxNumFiles, + zOrderByColumns) } else { - val useRepartition = - sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELTA_OPTIMIZE_REPARTITION_ENABLED) + val useRepartition = sparkSession.sessionState.conf.getConf( + DeltaSQLConf.DELTA_OPTIMIZE_REPARTITION_ENABLED) if (useRepartition) { input.repartition(numPartitions = 1) } else { @@ -369,7 +402,7 @@ class OptimizeExecutor( case other => throw new IllegalStateException( s"Unexpected action $other with type ${other.getClass}. File compaction job output" + - s"should only have AddFiles") + s"should only have AddFiles") } val removeFiles = bin.map(f => f.removeWithTimestamp(operationTimestamp, dataChange = false)) val updates = addFiles ++ removeFiles @@ -377,9 +410,9 @@ class OptimizeExecutor( } /** - * Attempts to commit the given actions to the log. In the case of a concurrent update, the given - * function will be invoked with a new transaction to allow custom conflict detection logic to - * indicate it is safe to try again, by returning `true`. + * Attempts to commit the given actions to the log. In the case of a concurrent update, + * the given function will be invoked with a new transaction to allow custom conflict + * detection logic to indicate it is safe to try again, by returning `true`. * * This function will continue to try to commit to the log as long as `f` returns `true`, * otherwise throws a subclass of [[ConcurrentModificationException]]. @@ -409,7 +442,8 @@ class OptimizeExecutor( private def createMetrics( sparkContext: SparkContext, addedFiles: Seq[AddFile], - removedFiles: Seq[RemoveFile]): Map[String, SQLMetric] = { + removedFiles: Seq[RemoveFile], + removedDVs: Seq[DeletionVectorDescriptor]): Map[String, SQLMetric] = { def setAndReturnMetric(description: String, value: Long) = { val metric = createMetric(sparkContext, description) @@ -419,19 +453,37 @@ class OptimizeExecutor( def totalSize(actions: Seq[FileAction]): Long = { var totalSize = 0L - actions.foreach { - file => - val fileSize = file match { - case addFile: AddFile => addFile.size - case removeFile: RemoveFile => removeFile.size.getOrElse(0L) - case default => - throw new IllegalArgumentException(s"Unknown FileAction type: ${default.getClass}") - } - totalSize += fileSize + actions.foreach { file => + val fileSize = file match { + case addFile: AddFile => addFile.size + case removeFile: RemoveFile => removeFile.size.getOrElse(0L) + case default => + throw new IllegalArgumentException(s"Unknown FileAction type: ${default.getClass}") + } + totalSize += fileSize } totalSize } + val (deletionVectorRowsRemoved, deletionVectorBytesRemoved) = + removedDVs.map(dv => (dv.cardinality, dv.sizeInBytes.toLong)) + .reduceLeftOption((dv1, dv2) => (dv1._1 + dv2._1, dv1._2 + dv2._2)) + .getOrElse((0L, 0L)) + + val dvMetrics: Map[String, SQLMetric] = Map( + "numDeletionVectorsRemoved" -> + setAndReturnMetric( + "total number of deletion vectors removed", + removedDVs.size), + "numDeletionVectorRowsRemoved" -> + setAndReturnMetric( + "total number of deletion vector rows removed", + deletionVectorRowsRemoved), + "numDeletionVectorBytesRemoved" -> + setAndReturnMetric( + "total number of bytes of removed deletion vectors", + deletionVectorBytesRemoved)) + val sizeStats = FileSizeStatsWithHistogram.create(addedFiles.map(_.size).sorted) Map[String, SQLMetric]( "minFileSize" -> setAndReturnMetric("minimum file size", sizeStats.get.min), @@ -444,6 +496,6 @@ class OptimizeExecutor( "numAddedBytes" -> setAndReturnMetric("total number of bytes added", totalSize(addedFiles)), "numRemovedBytes" -> setAndReturnMetric("total number of bytes removed", totalSize(removedFiles)) - ) + ) ++ dvMetrics } } diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/UpdateCommand.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/UpdateCommand.scala similarity index 66% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/UpdateCommand.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/UpdateCommand.scala index ad118470fc7f..94ccef961c08 100644 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/commands/UpdateCommand.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/UpdateCommand.scala @@ -14,32 +14,32 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.delta.commands +// scalastyle:off import.ordering.noEmptyLine +import org.apache.spark.sql.delta.{DeltaConfigs, DeltaLog, DeltaOperations, DeltaTableUtils, DeltaUDF, OptimisticTransaction} +import org.apache.spark.sql.delta.actions.{AddCDCFile, AddFile, FileAction} +import org.apache.spark.sql.delta.commands.cdc.CDCReader.{CDC_TYPE_COLUMN_NAME, CDC_TYPE_NOT_CDC, CDC_TYPE_UPDATE_POSTIMAGE, CDC_TYPE_UPDATE_PREIMAGE} +import org.apache.spark.sql.delta.files.{TahoeBatchFileIndex, TahoeFileIndex} +import org.apache.hadoop.fs.Path + import org.apache.spark.SparkContext -import org.apache.spark.sql._ +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, If, Literal} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.delta._ -import org.apache.spark.sql.delta.actions.{AddCDCFile, AddFile, FileAction} -import org.apache.spark.sql.delta.commands.cdc.CDCReader.{CDC_TYPE_COLUMN_NAME, CDC_TYPE_NOT_CDC, CDC_TYPE_UPDATE_POSTIMAGE, CDC_TYPE_UPDATE_PREIMAGE} -import org.apache.spark.sql.delta.files.{TahoeBatchFileIndex, TahoeFileIndex} -import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.LeafRunnableCommand -import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.metric.SQLMetrics.{createMetric, createTimingMetric} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.LongType -// scalastyle:off import.ordering.noEmptyLine -import org.apache.hadoop.fs.Path - /** * Gluten overwrite Delta: * - * This file is copied from Delta 2.2.0. It is modified to overcome the following issues: + * This file is copied from Delta 2.3.0. It is modified to overcome the following issues: * 1. In Clickhouse backend, we can't implement input_file_name() correctly, we can only implement * it so that it return a a list of filenames (concated by ','). */ @@ -47,18 +47,18 @@ import org.apache.hadoop.fs.Path /** * Performs an Update using `updateExpression` on the rows that match `condition` * - * Algorithm: 1) Identify the affected files, i.e., the files that may have the rows to be updated. - * 2) Scan affected files, apply the updates, and generate a new DF with updated rows. 3) Use the - * Delta protocol to atomically write the new DF as new files and remove the affected files that are - * identified in step 1. + * Algorithm: + * 1) Identify the affected files, i.e., the files that may have the rows to be updated. + * 2) Scan affected files, apply the updates, and generate a new DF with updated rows. + * 3) Use the Delta protocol to atomically write the new DF as new files and remove + * the affected files that are identified in step 1. */ case class UpdateCommand( tahoeFileIndex: TahoeFileIndex, target: LogicalPlan, updateExpressions: Seq[Expression], condition: Option[Expression]) - extends LeafRunnableCommand - with DeltaCommand { + extends LeafRunnableCommand with DeltaCommand { override val output: Seq[Attribute] = { Seq(AttributeReference("num_affected_rows", LongType)()) @@ -70,7 +70,9 @@ case class UpdateCommand( override lazy val metrics = Map[String, SQLMetric]( "numAddedFiles" -> createMetric(sc, "number of files added."), + "numAddedBytes" -> createMetric(sc, "number of bytes added"), "numRemovedFiles" -> createMetric(sc, "number of files removed."), + "numRemovedBytes" -> createMetric(sc, "number of bytes removed"), "numUpdatedRows" -> createMetric(sc, "number of rows updated."), "numCopiedRows" -> createMetric(sc, "number of rows copied."), "executionTimeMs" -> @@ -87,8 +89,14 @@ case class UpdateCommand( final override def run(sparkSession: SparkSession): Seq[Row] = { recordDeltaOperation(tahoeFileIndex.deltaLog, "delta.dml.update") { val deltaLog = tahoeFileIndex.deltaLog - deltaLog.assertRemovable() - deltaLog.withNewTransaction(txn => performUpdate(sparkSession, deltaLog, txn)) + deltaLog.withNewTransaction { txn => + DeltaLog.assertRemovable(txn.snapshot) + if (hasBeenExecuted(txn, sparkSession)) { + sendDriverMetrics(sparkSession, metrics) + return Seq.empty + } + performUpdate(sparkSession, deltaLog, txn) + } // Re-cache all cached plans(including this relation itself, if it's cached) that refer to // this data source relation. sparkSession.sharedState.cacheManager.recacheByPlan(sparkSession, target) @@ -97,13 +105,13 @@ case class UpdateCommand( } private def performUpdate( - sparkSession: SparkSession, - deltaLog: DeltaLog, - txn: OptimisticTransaction): Unit = { + sparkSession: SparkSession, deltaLog: DeltaLog, txn: OptimisticTransaction): Unit = { import org.apache.spark.sql.delta.implicits._ var numTouchedFiles: Long = 0 var numRewrittenFiles: Long = 0 + var numAddedBytes: Long = 0 + var numRemovedBytes: Long = 0 var numAddedChangeFiles: Long = 0 var changeFileBytes: Long = 0 var scanTimeMs: Long = 0 @@ -115,9 +123,7 @@ case class UpdateCommand( val updateCondition = condition.getOrElse(Literal.TrueLiteral) val (metadataPredicates, dataPredicates) = DeltaTableUtils.splitMetadataAndDataPredicates( - updateCondition, - txn.metadata.partitionColumns, - sparkSession) + updateCondition, txn.metadata.partitionColumns, sparkSession) val candidateFiles = txn.filterFiles(metadataPredicates ++ dataPredicates) val nameToAddFile = generateCandidateFileMap(deltaLog.dataPath, candidateFiles) @@ -134,34 +140,27 @@ case class UpdateCommand( } else { // Case 3: Find all the affected files using the user-specified condition val fileIndex = new TahoeBatchFileIndex( - sparkSession, - "update", - candidateFiles, - deltaLog, - tahoeFileIndex.path, - txn.snapshot) + sparkSession, "update", candidateFiles, deltaLog, tahoeFileIndex.path, txn.snapshot) // Keep everything from the resolved target except a new TahoeFileIndex // that only involves the affected files instead of all files. val newTarget = DeltaTableUtils.replaceFileIndex(target, fileIndex) val data = Dataset.ofRows(sparkSession, newTarget) val updatedRowCount = metrics("numUpdatedRows") - val updatedRowUdf = DeltaUDF - .boolean { - () => - updatedRowCount += 1 - true - } - .asNondeterministic() + val updatedRowUdf = DeltaUDF.boolean { () => + updatedRowCount += 1 + true + }.asNondeterministic() val pathsToRewrite = withStatusCode("DELTA", UpdateCommand.FINDING_TOUCHED_FILES_MSG) { - data - .filter(new Column(updateCondition)) + // --- modified start + data.filter(new Column(updateCondition)) .select(input_file_name().as("input_files")) .filter(updatedRowUdf()) .select(explode(split(col("input_files"), ","))) .distinct() .as[String] .collect() + // --- modified end } scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 @@ -177,13 +176,8 @@ case class UpdateCommand( } else { // Generate the new files containing the updated values withStatusCode("DELTA", UpdateCommand.rewritingFilesMsg(filesToRewrite.size)) { - rewriteFiles( - sparkSession, - txn, - tahoeFileIndex.path, - filesToRewrite.map(_.path), - nameToAddFile, - updateCondition) + rewriteFiles(sparkSession, txn, tahoeFileIndex.path, + filesToRewrite.map(_.path), nameToAddFile, updateCondition) } } @@ -191,6 +185,7 @@ case class UpdateCommand( val (changeActions, addActions) = newActions.partition(_.isInstanceOf[AddCDCFile]) numRewrittenFiles = addActions.size + numAddedBytes = addActions.map(_.getFileSize).sum numAddedChangeFiles = changeActions.size changeFileBytes = changeActions.collect { case f: AddCDCFile => f.size }.sum @@ -202,47 +197,42 @@ case class UpdateCommand( // files containing the updated values val operationTimestamp = System.currentTimeMillis() val deleteActions = filesToRewrite.map(_.removeWithTimestamp(operationTimestamp)) - + numRemovedBytes = filesToRewrite.map(_.getFileSize).sum deleteActions ++ newActions } - if (totalActions.nonEmpty) { - metrics("numAddedFiles").set(numRewrittenFiles) - metrics("numAddedChangeFiles").set(numAddedChangeFiles) - metrics("changeFileBytes").set(changeFileBytes) - metrics("numRemovedFiles").set(numTouchedFiles) - metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) - metrics("scanTimeMs").set(scanTimeMs) - metrics("rewriteTimeMs").set(rewriteTimeMs) - // In the case where the numUpdatedRows is not captured, we can siphon out the metrics from - // the BasicWriteStatsTracker. This is for case 2 where the update condition contains only - // metadata predicates and so the entire partition is re-written. - val outputRows = txn.getMetric("numOutputRows").map(_.value).getOrElse(-1L) - if ( - metrics("numUpdatedRows").value == 0 && outputRows != 0 && - metrics("numCopiedRows").value == 0 - ) { - // We know that numTouchedRows = numCopiedRows + numUpdatedRows. - // Since an entire partition was re-written, no rows were copied. - // So numTouchedRows == numUpdateRows - metrics("numUpdatedRows").set(metrics("numTouchedRows").value) - } else { - // This is for case 3 where the update condition contains both metadata and data predicates - // so relevant files will have some rows updated and some rows copied. We don't need to - // consider case 1 here, where no files match the update condition, as we know that - // `totalActions` is empty. - metrics("numCopiedRows").set( - metrics("numTouchedRows").value - metrics("numUpdatedRows").value) - } - txn.registerSQLMetrics(sparkSession, metrics) - txn.commit(totalActions, DeltaOperations.Update(condition.map(_.toString))) - // This is needed to make the SQL metrics visible in the Spark UI - val executionId = sparkSession.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates( - sparkSession.sparkContext, - executionId, - metrics.values.toSeq) + metrics("numAddedFiles").set(numRewrittenFiles) + metrics("numAddedBytes").set(numAddedBytes) + metrics("numAddedChangeFiles").set(numAddedChangeFiles) + metrics("changeFileBytes").set(changeFileBytes) + metrics("numRemovedFiles").set(numTouchedFiles) + metrics("numRemovedBytes").set(numRemovedBytes) + metrics("executionTimeMs").set((System.nanoTime() - startTime) / 1000 / 1000) + metrics("scanTimeMs").set(scanTimeMs) + metrics("rewriteTimeMs").set(rewriteTimeMs) + // In the case where the numUpdatedRows is not captured, we can siphon out the metrics from + // the BasicWriteStatsTracker. This is for case 2 where the update condition contains only + // metadata predicates and so the entire partition is re-written. + val outputRows = txn.getMetric("numOutputRows").map(_.value).getOrElse(-1L) + if (metrics("numUpdatedRows").value == 0 && outputRows != 0 && + metrics("numCopiedRows").value == 0) { + // We know that numTouchedRows = numCopiedRows + numUpdatedRows. + // Since an entire partition was re-written, no rows were copied. + // So numTouchedRows == numUpdateRows + metrics("numUpdatedRows").set(metrics("numTouchedRows").value) + } else { + // This is for case 3 where the update condition contains both metadata and data predicates + // so relevant files will have some rows updated and some rows copied. We don't need to + // consider case 1 here, where no files match the update condition, as we know that + // `totalActions` is empty. + metrics("numCopiedRows").set( + metrics("numTouchedRows").value - metrics("numUpdatedRows").value) } + txn.registerSQLMetrics(sparkSession, metrics) + + val finalActions = createSetTransaction(sparkSession, deltaLog).toSeq ++ totalActions + txn.commitIfNeeded(finalActions, DeltaOperations.Update(condition.map(_.toString))) + sendDriverMetrics(sparkSession, metrics) recordDeltaEvent( deltaLog, @@ -255,19 +245,17 @@ case class UpdateCommand( numAddedChangeFiles, changeFileBytes, scanTimeMs, - rewriteTimeMs - ) + rewriteTimeMs) ) } /** * Scan all the affected files and write out the updated files. * - * When CDF is enabled, includes the generation of CDC preimage and postimage columns for changed - * rows. + * When CDF is enabled, includes the generation of CDC preimage and postimage columns for + * changed rows. * - * @return - * the list of [[AddFile]]s and [[AddCDCFile]]s that have been written. + * @return the list of [[AddFile]]s and [[AddCDCFile]]s that have been written. */ private def rewriteFiles( spark: SparkSession, @@ -277,21 +265,18 @@ case class UpdateCommand( nameToAddFileMap: Map[String, AddFile], condition: Expression): Seq[FileAction] = { // Containing the map from the relative file path to AddFile - val baseRelation = - buildBaseRelation(spark, txn, "update", rootPath, inputLeafFiles, nameToAddFileMap) + val baseRelation = buildBaseRelation( + spark, txn, "update", rootPath, inputLeafFiles, nameToAddFileMap) val newTarget = DeltaTableUtils.replaceFileIndex(target, baseRelation.location) val targetDf = Dataset.ofRows(spark, newTarget) // Number of total rows that we have seen, i.e. are either copying or updating (sum of both). // This will be used later, along with numUpdatedRows, to determine numCopiedRows. val numTouchedRows = metrics("numTouchedRows") - val numTouchedRowsUdf = DeltaUDF - .boolean { - () => - numTouchedRows += 1 - true - } - .asNondeterministic() + val numTouchedRowsUdf = DeltaUDF.boolean { () => + numTouchedRows += 1 + true + }.asNondeterministic() val updatedDataFrame = UpdateCommand.withUpdatedColumns( target, @@ -300,8 +285,7 @@ case class UpdateCommand( targetDf .filter(numTouchedRowsUdf()) .withColumn(UpdateCommand.CONDITION_COLUMN_NAME, new Column(condition)), - UpdateCommand.shouldOutputCdc(txn) - ) + UpdateCommand.shouldOutputCdc(txn)) txn.writeFiles(updatedDataFrame) } @@ -324,25 +308,20 @@ object UpdateCommand { } /** - * Build the new columns. If the condition matches, generate the new value using the corresponding - * UPDATE EXPRESSION; otherwise, keep the original column value. + * Build the new columns. If the condition matches, generate the new value using + * the corresponding UPDATE EXPRESSION; otherwise, keep the original column value. * * When CDC is enabled, includes the generation of CDC pre-image and post-image columns for * changed rows. * - * @param target - * target we are updating into - * @param updateExpressions - * the update transformation to perform on the input DataFrame - * @param dfWithEvaluatedCondition - * source DataFrame on which we will apply the update expressions with an additional column - * CONDITION_COLUMN_NAME which is the true/false value of if the update condition is satisfied - * @param condition - * update condition - * @param shouldOutputCdc - * if we should output CDC data during this UPDATE operation. - * @return - * the updated DataFrame, with extra CDC columns if CDC is enabled + * @param target target we are updating into + * @param updateExpressions the update transformation to perform on the input DataFrame + * @param dfWithEvaluatedCondition source DataFrame on which we will apply the update expressions + * with an additional column CONDITION_COLUMN_NAME which is the + * true/false value of if the update condition is satisfied + * @param condition update condition + * @param shouldOutputCdc if we should output CDC data during this UPDATE operation. + * @return the updated DataFrame, with extra CDC columns if CDC is enabled */ def withUpdatedColumns( target: LogicalPlan, @@ -377,24 +356,22 @@ object UpdateCommand { If( UnresolvedAttribute(CONDITION_COLUMN_NAME), packedUpdates, // if it should be updated, then use `packagedUpdates` - array(struct(noopRewriteCols: _*)).expr - ) // else, this is a noop rewrite + array(struct(noopRewriteCols: _*)).expr) // else, this is a noop rewrite } // Explode the packed array, and project back out the final data columns. val finalColNames = target.output.map(_.name) :+ CDC_TYPE_COLUMN_NAME dfWithEvaluatedCondition .select(explode(new Column(packedData)).as("packedData")) - .select(finalColNames.map(n => col(s"packedData.`$n`").as(s"$n")): _*) + .select(finalColNames.map { n => col(s"packedData.`$n`").as(s"$n") }: _*) } else { - val finalCols = updateExpressions.zip(target.output).map { - case (update, original) => - val updated = if (condition == Literal.TrueLiteral) { - update - } else { - If(UnresolvedAttribute(CONDITION_COLUMN_NAME), update, original) - } - new Column(Alias(updated, original.name)()) + val finalCols = updateExpressions.zip(target.output).map { case (update, original) => + val updated = if (condition == Literal.TrueLiteral) { + update + } else { + If(UnresolvedAttribute(CONDITION_COLUMN_NAME), update, original) + } + new Column(Alias(updated, original.name)()) } dfWithEvaluatedCondition.select(finalCols: _*) @@ -407,25 +384,16 @@ object UpdateCommand { /** * Used to report details about update. * - * @param condition: - * what was the update condition - * @param numFilesTotal: - * how big is the table - * @param numTouchedFiles: - * how many files did we touch - * @param numRewrittenFiles: - * how many files had to be rewritten - * @param numAddedChangeFiles: - * how many change files were generated - * @param changeFileBytes: - * total size of change files generated - * @param scanTimeMs: - * how long did finding take - * @param rewriteTimeMs: - * how long did rewriting take + * @param condition: what was the update condition + * @param numFilesTotal: how big is the table + * @param numTouchedFiles: how many files did we touch + * @param numRewrittenFiles: how many files had to be rewritten + * @param numAddedChangeFiles: how many change files were generated + * @param changeFileBytes: total size of change files generated + * @param scanTimeMs: how long did finding take + * @param rewriteTimeMs: how long did rewriting take * - * @note - * All the time units are milliseconds. + * @note All the time units are milliseconds. */ case class UpdateMetric( condition: String, diff --git a/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/VacuumCommand.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/VacuumCommand.scala new file mode 100644 index 000000000000..5be548caf01c --- /dev/null +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/commands/VacuumCommand.scala @@ -0,0 +1,575 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.delta.commands + +// scalastyle:off import.ordering.noEmptyLine +import java.net.URI +import java.util.Date +import java.util.concurrent.TimeUnit + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.delta._ +import org.apache.spark.sql.delta.actions.{FileAction, RemoveFile} +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.util.DeltaFileOperations +import org.apache.spark.sql.delta.util.DeltaFileOperations.tryDeleteNonRecursive +import com.fasterxml.jackson.databind.annotation.JsonDeserialize +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.{Column, DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.execution.metric.SQLMetrics.createMetric +import org.apache.spark.sql.functions._ +import org.apache.spark.util.{Clock, SerializableConfiguration, SystemClock} + +/** + * Gluten overwrite Delta: + * + * This file is copied from Delta 2.3.0. It is modified to overcome the following issues: + * 1. In Gluten, part is a directory, but VacuumCommand assumes part is a file. So we need some + * modifications to make it work. + */ + +/** + * Vacuums the table by clearing all untracked files and folders within this table. + * First lists all the files and directories in the table, and gets the relative paths with + * respect to the base of the table. Then it gets the list of all tracked files for this table, + * which may or may not be within the table base path, and gets the relative paths of + * all the tracked files with respect to the base of the table. Files outside of the table path + * will be ignored. Then we take a diff of the files and delete directories that were already empty, + * and all files that are within the table that are no longer tracked. + */ +object VacuumCommand extends VacuumCommandImpl with Serializable { + + // --- modified start + case class FileNameAndSize(path: String, length: Long, isDir: Boolean = false) + // --- modified end + /** + * Additional check on retention duration to prevent people from shooting themselves in the foot. + */ + protected def checkRetentionPeriodSafety( + spark: SparkSession, + retentionMs: Option[Long], + configuredRetention: Long): Unit = { + require(retentionMs.forall(_ >= 0), "Retention for Vacuum can't be less than 0.") + val checkEnabled = + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_RETENTION_CHECK_ENABLED) + val retentionSafe = retentionMs.forall(_ >= configuredRetention) + var configuredRetentionHours = TimeUnit.MILLISECONDS.toHours(configuredRetention) + if (TimeUnit.HOURS.toMillis(configuredRetentionHours) < configuredRetention) { + configuredRetentionHours += 1 + } + require(!checkEnabled || retentionSafe, + s"""Are you sure you would like to vacuum files with such a low retention period? If you have + |writers that are currently writing to this table, there is a risk that you may corrupt the + |state of your Delta table. + | + |If you are certain that there are no operations being performed on this table, such as + |insert/upsert/delete/optimize, then you may turn off this check by setting: + |spark.databricks.delta.retentionDurationCheck.enabled = false + | + |If you are not sure, please use a value not less than "$configuredRetentionHours hours". + """.stripMargin) + } + + /** + * Clears all untracked files and folders within this table. First lists all the files and + * directories in the table, and gets the relative paths with respect to the base of the + * table. Then it gets the list of all tracked files for this table, which may or may not + * be within the table base path, and gets the relative paths of all the tracked files with + * respect to the base of the table. Files outside of the table path will be ignored. + * Then we take a diff of the files and delete directories that were already empty, and all files + * that are within the table that are no longer tracked. + * + * @param dryRun If set to true, no files will be deleted. Instead, we will list all files and + * directories that will be cleared. + * @param retentionHours An optional parameter to override the default Delta tombstone retention + * period + * @return A Dataset containing the paths of the files/folders to delete in dryRun mode. Otherwise + * returns the base path of the table. + */ + def gc( + spark: SparkSession, + deltaLog: DeltaLog, + dryRun: Boolean = true, + retentionHours: Option[Double] = None, + clock: Clock = new SystemClock): DataFrame = { + recordDeltaOperation(deltaLog, "delta.gc") { + + val path = deltaLog.dataPath + val deltaHadoopConf = deltaLog.newDeltaHadoopConf() + val fs = path.getFileSystem(deltaHadoopConf) + + import org.apache.spark.sql.delta.implicits._ + + val snapshot = deltaLog.update() + + require(snapshot.version >= 0, "No state defined for this table. Is this really " + + "a Delta table? Refusing to garbage collect.") + + // --- modified start + val isMergeTreeFormat = ClickHouseConfig + .isMergeTreeFormatEngine(deltaLog.unsafeVolatileMetadata.configuration) + // --- modified end + + DeletionVectorUtils.assertDeletionVectorsNotReadable( + spark, snapshot.metadata, snapshot.protocol) + + val snapshotTombstoneRetentionMillis = DeltaLog.tombstoneRetentionMillis(snapshot.metadata) + val retentionMillis = retentionHours.map(h => TimeUnit.HOURS.toMillis(math.round(h))) + checkRetentionPeriodSafety(spark, retentionMillis, snapshotTombstoneRetentionMillis) + + val deleteBeforeTimestamp = retentionMillis.map { millis => + clock.getTimeMillis() - millis + }.getOrElse(snapshot.minFileRetentionTimestamp) + // --- modified start: toGMTString is a deprecated function + logInfo(s"Starting garbage collection (dryRun = $dryRun) of untracked files older than " + + s"${new Date(deleteBeforeTimestamp).toString} in $path") + // --- modified end + val hadoopConf = spark.sparkContext.broadcast( + new SerializableConfiguration(deltaHadoopConf)) + val basePath = fs.makeQualified(path).toString + var isBloomFiltered = false + val parallelDeleteEnabled = + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_PARALLEL_DELETE_ENABLED) + val parallelDeletePartitions = + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_PARALLEL_DELETE_PARALLELISM) + .getOrElse(spark.sessionState.conf.numShufflePartitions) + val relativizeIgnoreError = + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_RELATIVIZE_IGNORE_ERROR) + val startTimeToIdentifyEligibleFiles = System.currentTimeMillis() + val validFiles = snapshot.stateDS + .mapPartitions { actions => + val reservoirBase = new Path(basePath) + val fs = reservoirBase.getFileSystem(hadoopConf.value.value) + actions.flatMap { + _.unwrap match { + case tombstone: RemoveFile if tombstone.delTimestamp < deleteBeforeTimestamp => + Nil + case fa: FileAction => + getValidRelativePathsAndSubdirs( + fa, + fs, + reservoirBase, + relativizeIgnoreError, + isBloomFiltered) + case _ => Nil + } + } + }.toDF("path") + + val partitionColumns = snapshot.metadata.partitionSchema.fieldNames + val parallelism = spark.sessionState.conf.parallelPartitionDiscoveryParallelism + + val allFilesAndDirs = DeltaFileOperations.recursiveListDirs( + spark, + Seq(basePath), + hadoopConf, + hiddenDirNameFilter = DeltaTableUtils.isHiddenDirectory(partitionColumns, _), + hiddenFileNameFilter = DeltaTableUtils.isHiddenDirectory(partitionColumns, _), + fileListingParallelism = Option(parallelism) + ) + .groupByKey(_.path) + .mapGroups { (k, v) => + val duplicates = v.toSeq + // of all the duplicates we can return the newest file. + duplicates.maxBy(_.modificationTime) + } + + try { + allFilesAndDirs.cache() + + implicit val fileNameAndSizeEncoder = org.apache.spark.sql.Encoders.product[FileNameAndSize] + + val dirCounts = allFilesAndDirs.where(col("isDir")).count() + 1 // +1 for the base path + + // 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 these + // --- modified start + val diff = if (isMergeTreeFormat) { + val diff_tmp = allFilesAndDirs + .where(col("modificationTime") < deleteBeforeTimestamp || col("isDir")) + .mapPartitions { fileStatusIterator => + val reservoirBase = new Path(basePath) + val fs = reservoirBase.getFileSystem(hadoopConf.value.value) + fileStatusIterator.flatMap { fileStatus => + if (fileStatus.isDir) { + Iterator.single(FileNameAndSize( + relativize(fileStatus.getHadoopPath, fs, reservoirBase, isDir = true), + 0L, + true)) + } else { + val dirs = getAllSubdirs(basePath, fileStatus.path, fs) + val dirsWithSlash = dirs.map { p => + val relativizedPath = relativize(new Path(p), fs, reservoirBase, isDir = true) + FileNameAndSize(relativizedPath, 0L, true) + } + dirsWithSlash ++ Iterator( + FileNameAndSize(relativize( + fileStatus.getHadoopPath, fs, reservoirBase, isDir = false), + fileStatus.length)) + } + } + } + .withColumn( + "dir", + when(col("isDir"), col("path")) + .otherwise(expr("substring_index(path, '/',size(split(path, '/')) -1)"))) + .groupBy(col("path"), col("dir")) + .agg(count(new Column("*")).as("count"), sum("length").as("length")) + + diff_tmp + .join(validFiles, diff_tmp("dir") === validFiles("path"), "leftanti") + .where(col("count") === 1) + } else { + allFilesAndDirs + .where(col("modificationTime") < deleteBeforeTimestamp || col("isDir")) + .mapPartitions { fileStatusIterator => + val reservoirBase = new Path(basePath) + val fs = reservoirBase.getFileSystem(hadoopConf.value.value) + fileStatusIterator.flatMap { fileStatus => + if (fileStatus.isDir) { + Iterator.single(FileNameAndSize( + relativize(fileStatus.getHadoopPath, fs, reservoirBase, isDir = true), 0L)) + } else { + val dirs = getAllSubdirs(basePath, fileStatus.path, fs) + val dirsWithSlash = dirs.map { p => + val relativizedPath = relativize(new Path(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(new Column("*")).as("count"), sum("length").as("length")) + .join(validFiles, Seq("path"), "leftanti") + .where(col("count") === 1) + } + // --- modified end + + val sizeOfDataToDeleteRow = diff.agg(sum("length").cast("long")).first + val sizeOfDataToDelete = if (sizeOfDataToDeleteRow.isNullAt(0)) { + 0L + } else { + sizeOfDataToDeleteRow.getLong(0) + } + + val diffFiles = 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)) + } + val timeTakenToIdentifyEligibleFiles = + System.currentTimeMillis() - startTimeToIdentifyEligibleFiles + + val numFiles = diffFiles.count() + if (dryRun) { + val stats = DeltaVacuumStats( + isDryRun = true, + specifiedRetentionMillis = retentionMillis, + defaultRetentionMillis = snapshotTombstoneRetentionMillis, + minRetainedTimestamp = deleteBeforeTimestamp, + dirsPresentBeforeDelete = dirCounts, + objectsDeleted = numFiles, + sizeOfDataToDelete = sizeOfDataToDelete, + timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles, + timeTakenForDelete = 0L) + + recordDeltaEvent(deltaLog, "delta.gc.stats", data = stats) + logConsole(s"Found $numFiles files ($sizeOfDataToDelete bytes) and directories in " + + s"a total of $dirCounts directories that are safe to delete.") + + return diffFiles.map(f => stringToPath(f).toString).toDF("path") + } + logVacuumStart( + spark, + deltaLog, + path, + diffFiles, + sizeOfDataToDelete, + retentionMillis, + snapshotTombstoneRetentionMillis) + + val deleteStartTime = System.currentTimeMillis() + val filesDeleted = try { + delete(diffFiles, spark, basePath, + hadoopConf, parallelDeleteEnabled, parallelDeletePartitions) + } catch { + case t: Throwable => + logVacuumEnd(deltaLog, spark, path) + throw t + } + val timeTakenForDelete = System.currentTimeMillis() - deleteStartTime + val stats = DeltaVacuumStats( + isDryRun = false, + specifiedRetentionMillis = retentionMillis, + defaultRetentionMillis = snapshotTombstoneRetentionMillis, + minRetainedTimestamp = deleteBeforeTimestamp, + dirsPresentBeforeDelete = dirCounts, + objectsDeleted = filesDeleted, + sizeOfDataToDelete = sizeOfDataToDelete, + timeTakenToIdentifyEligibleFiles = timeTakenToIdentifyEligibleFiles, + timeTakenForDelete = timeTakenForDelete) + recordDeltaEvent(deltaLog, "delta.gc.stats", data = stats) + logVacuumEnd(deltaLog, spark, path, Some(filesDeleted), Some(dirCounts)) + + + spark.createDataset(Seq(basePath)).toDF("path") + } finally { + allFilesAndDirs.unpersist() + } + } + } +} + +trait VacuumCommandImpl extends DeltaCommand { + + private val supportedFsForLogging = Seq( + "wasbs", "wasbss", "abfs", "abfss", "adl", "gs", "file", "hdfs" + ) + + /** + * Returns whether we should record vacuum metrics in the delta log. + */ + private def shouldLogVacuum( + spark: SparkSession, + deltaLog: DeltaLog, + hadoopConf: Configuration, + path: Path): Boolean = { + val logVacuumConf = spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_LOGGING_ENABLED) + + if (logVacuumConf.nonEmpty) { + return logVacuumConf.get + } + + val logStore = deltaLog.store + + try { + val rawResolvedUri: URI = logStore.resolvePathOnPhysicalStorage(path, hadoopConf).toUri + val scheme = rawResolvedUri.getScheme + supportedFsForLogging.contains(scheme) + } catch { + case _: UnsupportedOperationException => + logWarning("Vacuum event logging" + + " not enabled on this file system because we cannot detect your cloud storage type.") + false + } + } + + /** + * Record Vacuum specific metrics in the commit log at the START of vacuum. + * + * @param spark - spark session + * @param deltaLog - DeltaLog of the table + * @param path - the (data) path to the root of the table + * @param diff - the list of paths (files, directories) that are safe to delete + * @param sizeOfDataToDelete - the amount of data (bytes) to be deleted + * @param specifiedRetentionMillis - the optional override retention period (millis) to keep + * logically removed files before deleting them + * @param defaultRetentionMillis - the default retention period (millis) + */ + protected def logVacuumStart( + spark: SparkSession, + deltaLog: DeltaLog, + path: Path, + diff: Dataset[String], + sizeOfDataToDelete: Long, + specifiedRetentionMillis: Option[Long], + defaultRetentionMillis: Long): Unit = { + logInfo(s"Deleting untracked files and empty directories in $path. The amount of data to be " + + s"deleted is $sizeOfDataToDelete (in bytes)") + + // We perform an empty commit in order to record information about the Vacuum + if (shouldLogVacuum(spark, deltaLog, deltaLog.newDeltaHadoopConf(), path)) { + val checkEnabled = + spark.sessionState.conf.getConf(DeltaSQLConf.DELTA_VACUUM_RETENTION_CHECK_ENABLED) + val txn = deltaLog.startTransaction() + val metrics = Map[String, SQLMetric]( + "numFilesToDelete" -> createMetric(spark.sparkContext, "number of files to deleted"), + "sizeOfDataToDelete" -> createMetric(spark.sparkContext, + "The total amount of data to be deleted in bytes") + ) + metrics("numFilesToDelete").set(diff.count()) + metrics("sizeOfDataToDelete").set(sizeOfDataToDelete) + txn.registerSQLMetrics(spark, metrics) + txn.commit(actions = Seq(), DeltaOperations.VacuumStart( + checkEnabled, + specifiedRetentionMillis, + defaultRetentionMillis + )) + } + } + + /** + * Record Vacuum specific metrics in the commit log at the END of vacuum. + * + * @param deltaLog - DeltaLog of the table + * @param spark - spark session + * @param path - the (data) path to the root of the table + * @param filesDeleted - if the vacuum completed this will contain the number of files deleted. + * if the vacuum failed, this will be None. + * @param dirCounts - if the vacuum completed this will contain the number of directories + * vacuumed. if the vacuum failed, this will be None. + */ + protected def logVacuumEnd( + deltaLog: DeltaLog, + spark: SparkSession, + path: Path, + filesDeleted: Option[Long] = None, + dirCounts: Option[Long] = None): Unit = { + if (shouldLogVacuum(spark, deltaLog, deltaLog.newDeltaHadoopConf(), path)) { + val txn = deltaLog.startTransaction() + val status = if (filesDeleted.isEmpty && dirCounts.isEmpty) { "FAILED" } else { "COMPLETED" } + if (filesDeleted.nonEmpty && dirCounts.nonEmpty) { + val metrics = Map[String, SQLMetric]( + "numDeletedFiles" -> createMetric(spark.sparkContext, "number of files deleted."), + "numVacuumedDirectories" -> + createMetric(spark.sparkContext, "num of directories vacuumed."), + "status" -> createMetric(spark.sparkContext, "status of vacuum") + ) + metrics("numDeletedFiles").set(filesDeleted.get) + metrics("numVacuumedDirectories").set(dirCounts.get) + txn.registerSQLMetrics(spark, metrics) + } + txn.commit(actions = Seq(), DeltaOperations.VacuumEnd( + status + )) + } + + if (filesDeleted.nonEmpty) { + logConsole(s"Deleted ${filesDeleted.get} files and directories in a total " + + s"of ${dirCounts.get} directories.") + } + } + + /** + * Attempts to relativize the `path` with respect to the `reservoirBase` and converts the path to + * a string. + */ + protected def relativize( + path: Path, + fs: FileSystem, + reservoirBase: Path, + isDir: Boolean): String = { + pathToString(DeltaFileOperations.tryRelativizePath(fs, reservoirBase, path)) + } + + /** + * Wrapper function for DeltaFileOperations.getAllSubDirectories + * returns all subdirectories that `file` has with respect to `base`. + */ + protected def getAllSubdirs(base: String, file: String, fs: FileSystem): Iterator[String] = { + DeltaFileOperations.getAllSubDirectories(base, file)._1 + } + + /** + * Attempts to delete the list of candidate files. Returns the number of files deleted. + */ + protected def delete( + diff: Dataset[String], + spark: SparkSession, + basePath: String, + hadoopConf: Broadcast[SerializableConfiguration], + parallel: Boolean, + parallelPartitions: Int): Long = { + import org.apache.spark.sql.delta.implicits._ + + if (parallel) { + diff.repartition(parallelPartitions).mapPartitions { files => + val fs = new Path(basePath).getFileSystem(hadoopConf.value.value) + val filesDeletedPerPartition = + files.map(p => stringToPath(p)).count(f => tryDeleteNonRecursive(fs, f)) + Iterator(filesDeletedPerPartition) + }.collect().sum + } else { + val fs = new Path(basePath).getFileSystem(hadoopConf.value.value) + val fileResultSet = diff.toLocalIterator().asScala + fileResultSet.map(p => stringToPath(p)).count(f => tryDeleteNonRecursive(fs, f)) + } + } + + protected def stringToPath(path: String): Path = new Path(new URI(path)) + + protected def pathToString(path: Path): String = path.toUri.toString + + /** Returns the relative path of a file action or None if the file lives outside of the table. */ + protected def getActionRelativePath( + action: FileAction, + fs: FileSystem, + basePath: Path, + relativizeIgnoreError: Boolean): Option[String] = { + val filePath = stringToPath(action.path) + if (filePath.isAbsolute) { + val maybeRelative = + DeltaFileOperations.tryRelativizePath(fs, basePath, filePath, relativizeIgnoreError) + if (maybeRelative.isAbsolute) { + // This file lives outside the directory of the table. + None + } else { + Some(pathToString(maybeRelative)) + } + } else { + Some(pathToString(filePath)) + } + } + + + /** + * Returns the relative paths of all files and subdirectories for this action that must be + * retained during GC. + */ + protected def getValidRelativePathsAndSubdirs( + action: FileAction, + fs: FileSystem, + basePath: Path, + relativizeIgnoreError: Boolean, + isBloomFiltered: Boolean): Seq[String] = { + getActionRelativePath(action, fs, basePath, relativizeIgnoreError).map { relativePath => + Seq(relativePath) ++ getAllSubdirs("/", relativePath, fs) + }.getOrElse(Seq.empty) + } +} + +case class DeltaVacuumStats( + isDryRun: Boolean, + @JsonDeserialize(contentAs = classOf[java.lang.Long]) + specifiedRetentionMillis: Option[Long], + defaultRetentionMillis: Long, + minRetainedTimestamp: Long, + dirsPresentBeforeDelete: Long, + objectsDeleted: Long, + sizeOfDataToDelete: Long, + timeTakenToIdentifyEligibleFiles: Long, + timeTakenForDelete: Long) diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala similarity index 68% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala index 52fb921ad486..0a10d073e5c4 100644 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/delta/files/MergeTreeCommitProtocol.scala @@ -14,8 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ + package org.apache.spark.sql.delta.files +// scalastyle:off import.ordering.noEmptyLine +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.internal.io.FileCommitProtocol.TaskCommitMessage @@ -26,18 +30,28 @@ import org.apache.spark.sql.delta.commands.cdc.CDCReader.{CDC_LOCATION, CDC_PART import org.apache.spark.sql.delta.util.{DateFormatter, PartitionUtils, TimestampFormatter, Utils => DeltaUtils} import org.apache.spark.sql.types.StringType -// scalastyle:off import.ordering.noEmptyLine -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} - import java.util.UUID - import scala.collection.mutable.ArrayBuffer -class MergeTreeCommitProtocol(jobId: String, path: String, randomPrefixLength: Option[Int]) - extends FileCommitProtocol - with Serializable - with Logging { +/** + * This file is copied from the DelayedCommitProtocol of the Delta 2.3.0 + * and renamed to MergeTreeCommitProtocol. + * It is modified to overcome the following issues: + * 1. the function commitTask will return TaskCommitMessage(Nil), + * the FileStatus list will be get from the CH backend. + */ + +/** + * Writes out the files to `path` and returns a list of them in `addedStatuses`. Includes + * special handling for partitioning on [[CDC_PARTITION_COL]] for + * compatibility between enabled and disabled CDC; partitions with a value of false in this + * column produce no corresponding partitioning directory. + */ +class MergeTreeCommitProtocol( + jobId: String, + path: String, + randomPrefixLength: Option[Int]) + extends FileCommitProtocol with Serializable with Logging { // Track the list of files added by a task, only used on the executors. @transient protected var addedFiles: ArrayBuffer[(Map[String, String], String)] = _ @@ -58,11 +72,13 @@ class MergeTreeCommitProtocol(jobId: String, path: String, randomPrefixLength: O // Constants for CDC partition manipulation. Used only in newTaskTempFile(), but we define them // here to avoid building a new redundant regex for every file. - protected val cdcPartitionFalse = s"$CDC_PARTITION_COL=false" - protected val cdcPartitionTrue = s"$CDC_PARTITION_COL=true" + protected val cdcPartitionFalse = s"${CDC_PARTITION_COL}=false" + protected val cdcPartitionTrue = s"${CDC_PARTITION_COL}=true" protected val cdcPartitionTrueRegex = cdcPartitionTrue.r - override def setupJob(jobContext: JobContext): Unit = {} + override def setupJob(jobContext: JobContext): Unit = { + + } /** * Commits a job after the writes succeed. Must be called on the driver. Partitions the written @@ -70,8 +86,7 @@ class MergeTreeCommitProtocol(jobId: String, path: String, randomPrefixLength: O * by [[TransactionalWrite]] (i.e. AddFile's may have additional statistics injected) */ override def commitJob(jobContext: JobContext, taskCommits: Seq[TaskCommitMessage]): Unit = { - val (addFiles, changeFiles) = taskCommits - .flatMap(_.obj.asInstanceOf[Seq[_]]) + val (addFiles, changeFiles) = taskCommits.flatMap(_.obj.asInstanceOf[Seq[_]]) .partition { case _: AddFile => true case _: AddCDCFile => false @@ -128,12 +143,14 @@ class MergeTreeCommitProtocol(jobId: String, path: String, randomPrefixLength: O timestampFormatter) ._1 .get - parsedPartition.columnNames - .zip( - parsedPartition.literals - .map(l => Cast(l, StringType).eval()) - .map(Option(_).map(_.toString).orNull)) - .toMap + parsedPartition + .columnNames + .zip( + parsedPartition + .literals + .map(l => Cast(l, StringType).eval()) + .map(Option(_).map(_.toString).orNull)) + .toMap } /** @@ -142,56 +159,46 @@ class MergeTreeCommitProtocol(jobId: String, path: String, randomPrefixLength: O * * Includes special logic for CDC files and paths. Specifically, if the directory `dir` contains * the CDC partition `__is_cdc=true` then - * - the file name begins with `cdc-` instead of `part-` - * - the directory has the `__is_cdc=true` partition removed and is placed in the - * `_changed_data` folder + * - the file name begins with `cdc-` instead of `part-` + * - the directory has the `__is_cdc=true` partition removed and is placed in the `_changed_data` + * folder */ override def newTaskTempFile( - taskContext: TaskAttemptContext, - dir: Option[String], - ext: String): String = { + taskContext: TaskAttemptContext, dir: Option[String], ext: String): String = { val partitionValues = dir.map(parsePartitions).getOrElse(Map.empty[String, String]) val filename = getFileName(taskContext, ext, partitionValues) - val relativePath = randomPrefixLength - .map { - prefixLength => - DeltaUtils.getRandomPrefix(prefixLength) // Generate a random prefix as a first choice - } - .orElse { - dir // or else write into the partition directory if it is partitioned + val relativePath = randomPrefixLength.map { prefixLength => + DeltaUtils.getRandomPrefix(prefixLength) // Generate a random prefix as a first choice + }.orElse { + dir // or else write into the partition directory if it is partitioned + }.map { subDir => + // Do some surgery on the paths we write out to eliminate the CDC_PARTITION_COL. Non-CDC + // data is written to the base location, while CDC data is written to a special folder + // _change_data. + // The code here gets a bit complicated to accommodate two corner cases: an empty subdir + // can't be passed to new Path() at all, and a single-level subdir won't have a trailing + // slash. + if (subDir == cdcPartitionFalse) { + new Path(filename) + } else if (subDir.startsWith(cdcPartitionTrue)) { + val cleanedSubDir = cdcPartitionTrueRegex.replaceFirstIn(subDir, CDC_LOCATION) + new Path(cleanedSubDir, filename) + } else if (subDir.startsWith(cdcPartitionFalse)) { + // We need to remove the trailing slash in addition to the directory - otherwise + // it'll be interpreted as an absolute path and fail. + val cleanedSubDir = subDir.stripPrefix(cdcPartitionFalse + "/") + new Path(cleanedSubDir, filename) + } else { + new Path(subDir, filename) } - .map { - subDir => - // Do some surgery on the paths we write out to eliminate the CDC_PARTITION_COL. Non-CDC - // data is written to the base location, while CDC data is written to a special folder - // _change_data. - // The code here gets a bit complicated to accommodate two corner cases: an empty subdir - // can't be passed to new Path() at all, and a single-level subdir won't have a trailing - // slash. - if (subDir == cdcPartitionFalse) { - new Path(filename) - } else if (subDir.startsWith(cdcPartitionTrue)) { - val cleanedSubDir = cdcPartitionTrueRegex.replaceFirstIn(subDir, CDC_LOCATION) - new Path(cleanedSubDir, filename) - } else if (subDir.startsWith(cdcPartitionFalse)) { - // We need to remove the trailing slash in addition to the directory - otherwise - // it'll be interpreted as an absolute path and fail. - val cleanedSubDir = subDir.stripPrefix(cdcPartitionFalse + "/") - new Path(cleanedSubDir, filename) - } else { - new Path(subDir, filename) - } - } - .getOrElse(new Path(filename)) // or directly write out to the output path + }.getOrElse(new Path(filename)) // or directly write out to the output path addedFiles.append((partitionValues, relativePath.toUri.toString)) new Path(path, relativePath).toString } override def newTaskTempFileAbsPath( - taskContext: TaskAttemptContext, - absoluteDir: String, - ext: String): String = { + taskContext: TaskAttemptContext, absoluteDir: String, ext: String): String = { throw DeltaErrors.unsupportedAbsPathAddFile(s"$this") } @@ -213,8 +220,9 @@ class MergeTreeCommitProtocol(jobId: String, path: String, randomPrefixLength: O } override def commitTask(taskContext: TaskAttemptContext): TaskCommitMessage = { - if (addedFiles.nonEmpty) { - /* val fs = new Path(path, addedFiles.head._2).getFileSystem(taskContext.getConfiguration) + // --- modified start + /* if (addedFiles.nonEmpty) { + val fs = new Path(path, addedFiles.head._2).getFileSystem(taskContext.getConfiguration) val statuses: Seq[FileAction] = addedFiles.map { f => val filePath = new Path(path, new Path(new URI(f._2))) val stat = fs.getFileStatus(filePath) @@ -222,15 +230,15 @@ class MergeTreeCommitProtocol(jobId: String, path: String, randomPrefixLength: O buildActionFromAddedFile(f, stat, taskContext) }.toSeq - new TaskCommitMessage(statuses) */ - new TaskCommitMessage(Nil) + new TaskCommitMessage(statuses) } else { new TaskCommitMessage(Nil) - } + } */ + // --- modified end + new TaskCommitMessage(Nil) } override def abortTask(taskContext: TaskAttemptContext): Unit = { // TODO: we can also try delete the addedFiles as a best-effort cleanup. } - } diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/execution/datasources/v2/clickhouse/DeltaLogAdapter.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/execution/datasources/v2/clickhouse/DeltaLogAdapter.scala similarity index 100% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/execution/datasources/v2/clickhouse/DeltaLogAdapter.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/execution/datasources/v2/clickhouse/DeltaLogAdapter.scala diff --git a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/DeltaMergeTreeFileFormat.scala b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/DeltaMergeTreeFileFormat.scala similarity index 94% rename from backends-clickhouse/src/main/delta-22/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/DeltaMergeTreeFileFormat.scala rename to backends-clickhouse/src/main/delta-23/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/DeltaMergeTreeFileFormat.scala index 130790308db8..44e775b66688 100644 --- a/backends-clickhouse/src/main/delta-22/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/DeltaMergeTreeFileFormat.scala +++ b/backends-clickhouse/src/main/delta-23/org/apache/spark/sql/execution/datasources/v2/clickhouse/source/DeltaMergeTreeFileFormat.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.types.StructType import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} +@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass")) class DeltaMergeTreeFileFormat(metadata: Metadata) extends DeltaParquetFileFormat(metadata) { protected var database = "" @@ -75,7 +76,10 @@ class DeltaMergeTreeFileFormat(metadata: Metadata) extends DeltaParquetFileForma override def equals(other: Any): Boolean = { other match { case ff: DeltaMergeTreeFileFormat => - ff.columnMappingMode == columnMappingMode && ff.referenceSchema == referenceSchema + ff.columnMappingMode == columnMappingMode && + ff.referenceSchema == referenceSchema && + ff.isSplittable == isSplittable && + ff.disablePushDowns == disablePushDowns case _ => false } } diff --git a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala index 0aab14b78c87..a010a986cd4c 100644 --- a/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala +++ b/backends-clickhouse/src/main/scala/org/apache/gluten/backendsapi/clickhouse/CHTransformerApi.scala @@ -69,8 +69,6 @@ class CHTransformerApi extends TransformerApi with Logging { disableBucketedScan, filterExprs ) - case _: TahoeFileIndex => - throw new UnsupportedOperationException("Does not support delta-parquet") case _ => // Generate FilePartition for Parquet CHInputPartitionsUtil( diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala index e31560259720..d59467e11293 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/ClickhouseOptimisticTransaction.scala @@ -27,10 +27,11 @@ import org.apache.spark.sql.delta.constraints.{Constraint, Constraints} import org.apache.spark.sql.delta.files.MergeTreeCommitProtocol import org.apache.spark.sql.delta.schema.InvariantViolationException import org.apache.spark.sql.delta.sources.DeltaSQLConf -import org.apache.spark.sql.execution.SQLExecution +import org.apache.spark.sql.execution.{SparkPlan, SQLExecution} import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FakeRowAdaptor, FileFormatWriter, WriteJobStatsTracker} import org.apache.spark.sql.execution.datasources.v1.clickhouse.MergeTreeFileFormatWriter +import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.execution.datasources.v2.clickhouse.source.DeltaMergeTreeFileFormat import org.apache.spark.util.{Clock, SerializableConfiguration} @@ -51,156 +52,146 @@ class ClickhouseOptimisticTransaction( ) } + def insertFakeRowAdaptor(queryPlan: SparkPlan): SparkPlan = queryPlan match { + // if the child is columnar, we can just wrap&transfer the columnar data + case c2r: ColumnarToRowExecBase => + FakeRowAdaptor(c2r.child) + // If the child is aqe, we make aqe "support columnar", + // then aqe itself will guarantee to generate columnar outputs. + // So FakeRowAdaptor will always consumes columnar data, + // thus avoiding the case of c2r->aqe->r2c->writer + case aqe: AdaptiveSparkPlanExec => + FakeRowAdaptor( + AdaptiveSparkPlanExec( + aqe.inputPlan, + aqe.context, + aqe.preprocessingRules, + aqe.isSubquery, + supportsColumnar = true + )) + case other => FakeRowAdaptor(other) + } + override def writeFiles( inputData: Dataset[_], writeOptions: Option[DeltaOptions], additionalConstraints: Seq[Constraint]): Seq[FileAction] = { - hasWritten = true - - val spark = inputData.sparkSession - val (data, partitionSchema) = performCDCPartition(inputData) - val outputPath = deltaLog.dataPath - - val (queryExecution, output, generatedColumnConstraints, _) = - normalizeData(deltaLog, data) - val partitioningColumns = getPartitioningColumns(partitionSchema, output) - - val committer = new MergeTreeCommitProtocol("delta-mergetree", outputPath.toString, None) - - // val (optionalStatsTracker, _) = getOptionalStatsTrackerAndStatsCollection(output, outputPath, - // partitionSchema, data) - val (optionalStatsTracker, _) = (None, None) - - val constraints = - Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints - - SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) { - val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, output) - - val queryPlan = queryExecution.executedPlan - val newQueryPlan = queryPlan match { - // if the child is columnar, we can just wrap&transfer the columnar data - case c2r: ColumnarToRowExecBase => - FakeRowAdaptor(c2r.child) - // If the child is aqe, we make aqe "support columnar", - // then aqe itself will guarantee to generate columnar outputs. - // So FakeRowAdaptor will always consumes columnar data, - // thus avoiding the case of c2r->aqe->r2c->writer - case aqe: AdaptiveSparkPlanExec => - FakeRowAdaptor( - AdaptiveSparkPlanExec( - aqe.inputPlan, - aqe.context, - aqe.preprocessingRules, - aqe.isSubquery, - supportsColumnar = true - )) - case other => FakeRowAdaptor(other) - } - - val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() - - if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { - val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker( - new SerializableConfiguration(deltaLog.newDeltaHadoopConf()), - BasicWriteJobStatsTracker.metrics) -// registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) - statsTrackers.append(basicWriteJobStatsTracker) - } - - // Retain only a minimal selection of Spark writer options to avoid any potential - // compatibility issues - var options = writeOptions match { - case None => Map.empty[String, String] - case Some(writeOptions) => - writeOptions.options.filterKeys { - key => - key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) || - key.equalsIgnoreCase(DeltaOptions.COMPRESSION) - }.toMap - } - - spark.conf.getAll.foreach( - entry => { - if ( - entry._1.startsWith(s"${CHBackendSettings.getBackendConfigPrefix}.runtime_settings") - || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) - ) { - options += (entry._1 -> entry._2) - } - }) - - try { - val tableV2 = ClickHouseTableV2.getTable(deltaLog) - MergeTreeFileFormatWriter.write( - sparkSession = spark, - plan = newQueryPlan, - fileFormat = new DeltaMergeTreeFileFormat( - metadata, - tableV2.dataBaseName, - tableV2.tableName, - ClickhouseSnapshot.genSnapshotId(tableV2.snapshot), - tableV2.orderByKeyOption, - tableV2.lowCardKeyOption, - tableV2.minmaxIndexKeyOption, - tableV2.bfIndexKeyOption, - tableV2.setIndexKeyOption, - tableV2.primaryKeyOption, - tableV2.clickhouseTableConfigs, - tableV2.partitionColumns - ), - // formats. - committer = committer, - outputSpec = outputSpec, - // scalastyle:off deltahadoopconfiguration - hadoopConf = - spark.sessionState.newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options), - // scalastyle:on deltahadoopconfiguration - orderByKeyOption = tableV2.orderByKeyOption, - lowCardKeyOption = tableV2.lowCardKeyOption, - minmaxIndexKeyOption = tableV2.minmaxIndexKeyOption, - bfIndexKeyOption = tableV2.bfIndexKeyOption, - setIndexKeyOption = tableV2.setIndexKeyOption, - primaryKeyOption = tableV2.primaryKeyOption, - partitionColumns = partitioningColumns, - bucketSpec = tableV2.bucketOption, - statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, - options = options, - constraints = constraints - ) - } catch { - case s: SparkException => - // Pull an InvariantViolationException up to the top level if it was the root cause. - val violationException = ExceptionUtils.getRootCause(s) - if (violationException.isInstanceOf[InvariantViolationException]) { - throw violationException - } else { - throw s - } + if (ClickHouseConfig.isMergeTreeFormatEngine(metadata.configuration)) { + hasWritten = true + + val spark = inputData.sparkSession + val (data, partitionSchema) = performCDCPartition(inputData) + val outputPath = deltaLog.dataPath + + val (queryExecution, output, generatedColumnConstraints, _) = + normalizeData(deltaLog, data) + val partitioningColumns = getPartitioningColumns(partitionSchema, output) + + val committer = new MergeTreeCommitProtocol("delta-mergetree", outputPath.toString, None) + + // val (optionalStatsTracker, _) = + // getOptionalStatsTrackerAndStatsCollection(output, outputPath, partitionSchema, data) + val (optionalStatsTracker, _) = (None, None) + + val constraints = + Constraints.getAll(metadata, spark) ++ generatedColumnConstraints ++ additionalConstraints + + SQLExecution.withNewExecutionId(queryExecution, Option("deltaTransactionalWrite")) { + val outputSpec = FileFormatWriter.OutputSpec(outputPath.toString, Map.empty, output) + + val queryPlan = queryExecution.executedPlan + val newQueryPlan = insertFakeRowAdaptor(queryPlan) + + val statsTrackers: ListBuffer[WriteJobStatsTracker] = ListBuffer() + + if (spark.conf.get(DeltaSQLConf.DELTA_HISTORY_METRICS_ENABLED)) { + val basicWriteJobStatsTracker = new BasicWriteJobStatsTracker( + new SerializableConfiguration(deltaLog.newDeltaHadoopConf()), + BasicWriteJobStatsTracker.metrics) + // registerSQLMetrics(spark, basicWriteJobStatsTracker.driverSideMetrics) + statsTrackers.append(basicWriteJobStatsTracker) + } + + // Retain only a minimal selection of Spark writer options to avoid any potential + // compatibility issues + var options = writeOptions match { + case None => Map.empty[String, String] + case Some(writeOptions) => + writeOptions.options.filterKeys { + key => + key.equalsIgnoreCase(DeltaOptions.MAX_RECORDS_PER_FILE) || + key.equalsIgnoreCase(DeltaOptions.COMPRESSION) + }.toMap + } + + spark.conf.getAll.foreach( + entry => { + if ( + entry._1.startsWith(s"${CHBackendSettings.getBackendConfigPrefix}.runtime_settings") + || entry._1.equalsIgnoreCase(DeltaSQLConf.DELTA_OPTIMIZE_MIN_FILE_SIZE.key) + ) { + options += (entry._1 -> entry._2) + } + }) + + try { + val tableV2 = ClickHouseTableV2.getTable(deltaLog) + MergeTreeFileFormatWriter.write( + sparkSession = spark, + plan = newQueryPlan, + fileFormat = new DeltaMergeTreeFileFormat( + metadata, + tableV2.dataBaseName, + tableV2.tableName, + ClickhouseSnapshot.genSnapshotId(tableV2.snapshot), + tableV2.orderByKeyOption, + tableV2.lowCardKeyOption, + tableV2.minmaxIndexKeyOption, + tableV2.bfIndexKeyOption, + tableV2.setIndexKeyOption, + tableV2.primaryKeyOption, + tableV2.clickhouseTableConfigs, + tableV2.partitionColumns + ), + // formats. + committer = committer, + outputSpec = outputSpec, + // scalastyle:off deltahadoopconfiguration + hadoopConf = spark.sessionState + .newHadoopConfWithOptions(metadata.configuration ++ deltaLog.options), + // scalastyle:on deltahadoopconfiguration + orderByKeyOption = tableV2.orderByKeyOption, + lowCardKeyOption = tableV2.lowCardKeyOption, + minmaxIndexKeyOption = tableV2.minmaxIndexKeyOption, + bfIndexKeyOption = tableV2.bfIndexKeyOption, + setIndexKeyOption = tableV2.setIndexKeyOption, + primaryKeyOption = tableV2.primaryKeyOption, + partitionColumns = partitioningColumns, + bucketSpec = tableV2.bucketOption, + statsTrackers = optionalStatsTracker.toSeq ++ statsTrackers, + options = options, + constraints = constraints + ) + } catch { + case s: SparkException => + // Pull an InvariantViolationException up to the top level if it was the root cause. + val violationException = ExceptionUtils.getRootCause(s) + if (violationException.isInstanceOf[InvariantViolationException]) { + throw violationException + } else { + throw s + } + } } + committer.addedStatuses.toSeq ++ committer.changeFiles + } else { + // TODO: support native delta parquet write + // 1. insert FakeRowAdaptor + // 2. DeltaInvariantCheckerExec transform + // 3. DeltaTaskStatisticsTracker collect null count / min values / max values + // 4. set the parameters 'staticPartitionWriteOnly', 'isNativeAppliable', + // 'nativeFormat' in the LocalProperty of the sparkcontext + super.writeFiles(inputData, writeOptions, additionalConstraints) } - - // val resultFiles = committer.addedStatuses - // .map { - // a => - // a.copy(stats = optionalStatsTracker - // .map(_.recordedStats(new Path(new URI(a.path)).getName)) - // .getOrElse(a.stats)) - // } - /* - .filter { - // In some cases, we can write out an empty `inputData`. - // Some examples of this (though, they - // may be fixed in the future) are the MERGE command when you delete with empty source, or - // empty target, or on disjoint tables. This is hard to catch before the write without - // collecting the DF ahead of time. Instead, we can return only the AddFiles that - // a) actually add rows, or - // b) don't have any stats so we don't know the number of rows at all - case a: AddFile => a.numLogicalRecords.forall(_ > 0) - case _ => true - } - */ - - committer.addedStatuses.toSeq ++ committer.changeFiles } } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala index 1107c6a2ef76..d5cd4f984ca6 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/catalog/ClickHouseTableV2.scala @@ -19,7 +19,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable} import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.connector.catalog.TableCatalog import org.apache.spark.sql.connector.read.InputPartition import org.apache.spark.sql.connector.write.{LogicalWriteInfo, WriteBuilder} import org.apache.spark.sql.delta.{ClickhouseSnapshot, DeltaErrors, DeltaLog, DeltaTimeTravelSpec} @@ -28,7 +27,6 @@ import org.apache.spark.sql.delta.catalog.ClickHouseTableV2.deltaLog2Table import org.apache.spark.sql.delta.sources.DeltaDataSource import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, PartitionDirectory} import org.apache.spark.sql.execution.datasources.utils.MergeTreePartsPartitionsUtil -import org.apache.spark.sql.execution.datasources.v2.clickhouse.ClickHouseConfig import org.apache.spark.sql.execution.datasources.v2.clickhouse.source.DeltaMergeTreeFileFormat import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.collection.BitSet @@ -83,7 +81,6 @@ class ClickHouseTableV2( override def properties(): ju.Map[String, String] = { val ret = super.properties() - ret.put(TableCatalog.PROP_PROVIDER, ClickHouseConfig.NAME) // for file path based write if (snapshot.version < 0 && clickhouseExtensionOptions.nonEmpty) { @@ -232,6 +229,7 @@ class ClickHouseTableV2( partitionColumns ) } + def cacheThis(): Unit = { deltaLog2Table.put(deltaLog, this) } diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala index 7af1abe43722..f7a180b6a239 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommandOverwrites.scala @@ -25,6 +25,7 @@ import org.apache.spark.internal.io.SparkHadoopWriterUtils import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog.CatalogTableType import org.apache.spark.sql.delta._ import org.apache.spark.sql.delta.actions.{AddFile, FileAction} import org.apache.spark.sql.delta.catalog.ClickHouseTableV2 @@ -243,21 +244,6 @@ object OptimizeTableCommandOverwrites extends Logging { } - private def isDeltaTable(spark: SparkSession, tableName: TableIdentifier): Boolean = { - val catalog = spark.sessionState.catalog - val tableIsNotTemporaryTable = !catalog.isTempView(tableName) - val tableExists = { - (tableName.database.isEmpty || catalog.databaseExists(tableName.database.get)) && - catalog.tableExists(tableName) - } - tableIsNotTemporaryTable && tableExists && catalog - .getTableMetadata(tableName) - .provider - .get - .toLowerCase() - .equals("clickhouse") - } - def getDeltaLogClickhouse( spark: SparkSession, path: Option[String], @@ -276,7 +262,17 @@ object OptimizeTableCommandOverwrites extends Logging { } else if (CHDataSourceUtils.isClickHouseTable(spark, tableIdentifier.get)) { new Path(metadata.location) } else { - throw DeltaErrors.notADeltaTableException(operationName) + DeltaTableIdentifier(spark, tableIdentifier.get) match { + case Some(id) if id.path.nonEmpty => + new Path(id.path.get) + case Some(id) if id.table.nonEmpty => + new Path(metadata.location) + case _ => + if (metadata.tableType == CatalogTableType.VIEW) { + throw DeltaErrors.viewNotSupported(operationName) + } + throw DeltaErrors.notADeltaTableException(operationName) + } } } else { throw DeltaErrors.missingTableIdentifierException(operationName) diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala index e7eb3aeb05d0..232e9ec10c5b 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseConfig.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.execution.datasources.v2.clickhouse import org.apache.spark.sql.catalyst.catalog.BucketSpec -import org.apache.spark.sql.connector.catalog.TableCatalog import java.util @@ -29,7 +28,8 @@ object ClickHouseConfig { val NAME = "clickhouse" val ALT_NAME = "clickhouse" val METADATA_DIR = "_delta_log" - val DEFAULT_ENGINE = "MergeTree" + val FORMAT_ENGINE = "engine" + val DEFAULT_ENGINE = "mergetree" val OPT_NAME_PREFIX = "clickhouse." @deprecated @@ -47,16 +47,15 @@ object ClickHouseConfig { buckets: Option[BucketSpec] = None): Map[String, String] = { val configurations = scala.collection.mutable.Map[String, String]() allProperties.asScala.foreach(configurations += _) - configurations.put(TableCatalog.PROP_PROVIDER, ClickHouseConfig.NAME) if (!configurations.contains("metadata_path")) { configurations += ("metadata_path" -> METADATA_DIR) } - if (!configurations.contains("engine")) { - configurations += ("engine" -> DEFAULT_ENGINE) + if (!configurations.contains(FORMAT_ENGINE)) { + configurations += (FORMAT_ENGINE -> DEFAULT_ENGINE) } else { - val engineValue = configurations.get("engine") + val engineValue = configurations.get(FORMAT_ENGINE) if (!engineValue.equals(DEFAULT_ENGINE) && !engineValue.equals("parquet")) { - configurations += ("engine" -> DEFAULT_ENGINE) + configurations += (FORMAT_ENGINE -> DEFAULT_ENGINE) } } if (!configurations.contains("sampling_key")) { @@ -80,6 +79,11 @@ object ClickHouseConfig { configurations.toMap } + def isMergeTreeFormatEngine(configuration: Map[String, String]): Boolean = { + configuration.contains(FORMAT_ENGINE) && + configuration.get(FORMAT_ENGINE).get.equals(DEFAULT_ENGINE) + } + /** Get the related clickhouse option when using DataFrameWriter / DataFrameReader */ def getMergeTreeConfigurations( properties: util.Map[String, String] diff --git a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala index c2d208df5ed9..61e1da44d0af 100644 --- a/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala +++ b/backends-clickhouse/src/main/scala/org/apache/spark/sql/execution/datasources/v2/clickhouse/ClickHouseSparkCatalog.scala @@ -15,6 +15,7 @@ * limitations under the License. */ package org.apache.spark.sql.execution.datasources.v2.clickhouse + import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} @@ -26,12 +27,12 @@ import org.apache.spark.sql.connector.catalog._ import org.apache.spark.sql.connector.catalog.TableCapability.V1_BATCH_WRITE import org.apache.spark.sql.connector.expressions.Transform import org.apache.spark.sql.connector.write.{LogicalWriteInfo, V1Write, WriteBuilder} -import org.apache.spark.sql.delta.{DeltaErrors, DeltaLog, DeltaOptions} +import org.apache.spark.sql.delta.{DeltaConfigs, DeltaErrors, DeltaLog, DeltaOptions, DeltaTableUtils} import org.apache.spark.sql.delta.DeltaTableIdentifier.gluePermissionError -import org.apache.spark.sql.delta.catalog.{ClickHouseTableV2, TempClickHouseTableV2} +import org.apache.spark.sql.delta.catalog.{ClickHouseTableV2, DeltaTableV2, TempClickHouseTableV2} import org.apache.spark.sql.delta.commands.{CreateDeltaTableCommand, TableCreationModes, WriteIntoDelta} import org.apache.spark.sql.delta.metering.DeltaLogging -import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.sources.{DeltaSourceUtils, DeltaSQLConf} import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} import org.apache.spark.sql.execution.datasources.v2.clickhouse.utils.CHDataSourceUtils import org.apache.spark.sql.sources.InsertableRelation @@ -52,6 +53,15 @@ class ClickHouseSparkCatalog val spark = SparkSession.active + private def createCatalogTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String] + ): Table = { + super.createTable(ident, schema, partitions, properties) + } + override def createTable( ident: Identifier, schema: StructType, @@ -66,8 +76,18 @@ class ClickHouseSparkCatalog Map.empty, sourceQuery = None, TableCreationModes.Create) + } else if (DeltaSourceUtils.isDeltaDataSourceName(getProvider(properties))) { + createDeltaTable( + ident, + schema, + partitions, + properties, + Map.empty, + sourceQuery = None, + TableCreationModes.Create + ) } else { - super.createTable(ident, schema, partitions, properties) + createCatalogTable(ident, schema, partitions, properties) } } @@ -120,7 +140,10 @@ class ClickHouseSparkCatalog .copy(locationUri = locUriOpt) val tableType = if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED - val id = TableIdentifier(ident.name(), ident.namespace().lastOption) + val id = { + TableIdentifier(ident.name(), ident.namespace().lastOption) + } + val existingTableOpt = getExistingTableIfExists(id) val loc = new Path(locUriOpt.getOrElse(spark.sessionState.catalog.defaultTablePath(id))) val commentOpt = Option(allTableProperties.get("comment")) @@ -136,7 +159,7 @@ class ClickHouseSparkCatalog comment = commentOpt ) - val withDb = verifyTableAndSolidify(tableDesc, None) + val withDb = verifyTableAndSolidify(tableDesc, None, true) val writer = sourceQuery.map { df => @@ -156,7 +179,7 @@ class ClickHouseSparkCatalog CreateDeltaTableCommand( withDb, - getExistingTableIfExists(tableDesc), + existingTableOpt, operation.mode, writer, operation = operation, @@ -166,14 +189,134 @@ class ClickHouseSparkCatalog } logInfo(s"create table ${ident.toString} successfully.") - val loadedNewTable = loadTable(ident) - loadedNewTable + loadTable(ident) + } + + /** + * Creates a Delta table + * + * @param ident + * The identifier of the table + * @param schema + * The schema of the table + * @param partitions + * The partition transforms for the table + * @param allTableProperties + * The table properties that configure the behavior of the table or provide information about + * the table + * @param writeOptions + * Options specific to the write during table creation or replacement + * @param sourceQuery + * A query if this CREATE request came from a CTAS or RTAS + * @param operation + * The specific table creation mode, whether this is a Create/Replace/Create or Replace + */ + private def createDeltaTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + allTableProperties: util.Map[String, String], + writeOptions: Map[String, String], + sourceQuery: Option[DataFrame], + operation: TableCreationModes.CreationMode + ): Table = { + // These two keys are tableProperties in data source v2 but not in v1, so we have to filter + // them out. Otherwise property consistency checks will fail. + val tableProperties = allTableProperties.asScala.filterKeys { + case TableCatalog.PROP_LOCATION => false + case TableCatalog.PROP_PROVIDER => false + case TableCatalog.PROP_COMMENT => false + case TableCatalog.PROP_OWNER => false + case TableCatalog.PROP_EXTERNAL => false + case "path" => false + case _ => true + }.toMap + val (partitionColumns, maybeBucketSpec) = + SparkShimLoader.getSparkShims.convertPartitionTransforms(partitions) + var newSchema = schema + var newPartitionColumns = partitionColumns + var newBucketSpec = maybeBucketSpec + val conf = spark.sessionState.conf + + val isByPath = isPathIdentifier(ident) + if ( + isByPath && !conf.getConf(DeltaSQLConf.DELTA_LEGACY_ALLOW_AMBIGUOUS_PATHS) + && allTableProperties.containsKey("location") + // The location property can be qualified and different from the path in the identifier, so + // we check `endsWith` here. + && Option(allTableProperties.get("location")).exists(!_.endsWith(ident.name())) + ) { + throw DeltaErrors.ambiguousPathsInCreateTableException( + ident.name(), + allTableProperties.get("location")) + } + val location = if (isByPath) { + Option(ident.name()) + } else { + Option(allTableProperties.get("location")) + } + val id = { + TableIdentifier(ident.name(), ident.namespace().lastOption) + } + var locUriOpt = location.map(CatalogUtils.stringToURI) + val existingTableOpt = getExistingTableIfExists(id) + val loc = locUriOpt + .orElse(existingTableOpt.flatMap(_.storage.locationUri)) + .getOrElse(spark.sessionState.catalog.defaultTablePath(id)) + val storage = DataSource + .buildStorageFormatFromOptions(writeOptions) + .copy(locationUri = Option(loc)) + val tableType = + if (location.isDefined) CatalogTableType.EXTERNAL else CatalogTableType.MANAGED + val commentOpt = Option(allTableProperties.get("comment")) + + var tableDesc = new CatalogTable( + identifier = id, + tableType = tableType, + storage = storage, + schema = newSchema, + provider = Some(DeltaSourceUtils.ALT_NAME), + partitionColumnNames = newPartitionColumns, + bucketSpec = newBucketSpec, + properties = tableProperties, + comment = commentOpt + ) + + val withDb = verifyTableAndSolidify(tableDesc, None) + + val writer = sourceQuery.map { + df => + WriteIntoDelta( + DeltaLog.forTable(spark, new Path(loc)), + operation.mode, + new DeltaOptions(withDb.storage.properties, spark.sessionState.conf), + withDb.partitionColumnNames, + withDb.properties ++ commentOpt.map("comment" -> _), + df, + schemaInCatalog = if (newSchema != schema) Some(newSchema) else None + ) + } + + CreateDeltaTableCommand( + withDb, + existingTableOpt, + operation.mode, + writer, + operation, + tableByPath = isByPath).run(spark) + + loadTable(ident) } /** Performs checks on the parameters provided for table creation for a ClickHouse table. */ private def verifyTableAndSolidify( tableDesc: CatalogTable, - query: Option[LogicalPlan]): CatalogTable = { + query: Option[LogicalPlan], + isMergeTree: Boolean = false): CatalogTable = { + + if (!isMergeTree && tableDesc.bucketSpec.isDefined) { + throw DeltaErrors.operationNotSupportedException("Bucketing", tableDesc.identifier) + } val schema = query .map { @@ -189,30 +332,36 @@ class ClickHouseSparkCatalog caseSensitive = false ) // Delta is case insensitive + val validatedConfigurations = if (isMergeTree) { + tableDesc.properties + } else { + DeltaConfigs.validateConfigurations(tableDesc.properties) + } + val db = tableDesc.identifier.database.getOrElse(catalog.getCurrentDatabase) val tableIdentWithDB = tableDesc.identifier.copy(database = Some(db)) tableDesc.copy( identifier = tableIdentWithDB, schema = schema, - properties = tableDesc.properties) + properties = validatedConfigurations) } /** Checks if a table already exists for the provided identifier. */ - private def getExistingTableIfExists(table: CatalogTable): Option[CatalogTable] = { + def getExistingTableIfExists(table: TableIdentifier): Option[CatalogTable] = { // If this is a path identifier, we cannot return an existing CatalogTable. The Create command // will check the file system itself if (isPathIdentifier(table)) return None - val tableExists = catalog.tableExists(table.identifier) + val tableExists = catalog.tableExists(table) if (tableExists) { - val oldTable = catalog.getTableMetadata(table.identifier) + val oldTable = catalog.getTableMetadata(table) if (oldTable.tableType == CatalogTableType.VIEW) { - throw new AnalysisException( - s"${table.identifier} is a view. You may not write data into a view.") + throw new AnalysisException(s"$table is a view. You may not write data into a view.") } - if (!CHDataSourceUtils.isClickHouseTable(oldTable.provider)) { - throw new AnalysisException( - s"${table.identifier} is not a ClickHouse table. Please drop " + - s"this table first if you would like to recreate it.") + if ( + !DeltaSourceUtils.isDeltaTable(oldTable.provider) && + !CHDataSourceUtils.isClickHouseTable(oldTable.provider) + ) { + throw DeltaErrors.notADeltaTable(table.table) } Some(oldTable) } else { @@ -233,6 +382,12 @@ class ClickHouseSparkCatalog new Path(v1.catalogTable.location), catalogTable = Some(v1.catalogTable), tableIdentifier = Some(ident.toString)) + case v1: V1Table if DeltaTableUtils.isDeltaTable(v1.catalogTable) => + DeltaTableV2( + spark, + new Path(v1.catalogTable.location), + catalogTable = Some(v1.catalogTable), + tableIdentifier = Some(ident.toString)) case o => o } @@ -249,8 +404,12 @@ class ClickHouseSparkCatalog } } - private def newDeltaPathTable(ident: Identifier): ClickHouseTableV2 = { - new ClickHouseTableV2(spark, new Path(ident.name())) + private def newDeltaPathTable(ident: Identifier): DeltaTableV2 = { + if (hasClickHouseNamespace(ident)) { + new ClickHouseTableV2(spark, new Path(ident.name())) + } else { + DeltaTableV2(spark, new Path(ident.name())) + } } /** support to delete mergetree data from the external table */ @@ -284,11 +443,15 @@ class ClickHouseSparkCatalog partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = recordFrameProfile("DeltaCatalog", "stageReplace") { - if (CHDataSourceUtils.isClickHouseDataSourceName(getProvider(properties))) { + if ( + CHDataSourceUtils.isClickHouseDataSourceName(getProvider(properties)) || + DeltaSourceUtils.isDeltaDataSourceName(getProvider(properties)) + ) { new StagedDeltaTableV2(ident, schema, partitions, properties, TableCreationModes.Replace) } else { super.dropTable(ident) - BestEffortStagedTable(ident, super.createTable(ident, schema, partitions, properties), this) + val table = createCatalogTable(ident, schema, partitions, properties) + BestEffortStagedTable(ident, table, this) } } @@ -298,7 +461,10 @@ class ClickHouseSparkCatalog partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = recordFrameProfile("DeltaCatalog", "stageCreateOrReplace") { - if (CHDataSourceUtils.isClickHouseDataSourceName(getProvider(properties))) { + if ( + CHDataSourceUtils.isClickHouseDataSourceName(getProvider(properties)) || + DeltaSourceUtils.isDeltaDataSourceName(getProvider(properties)) + ) { new StagedDeltaTableV2( ident, schema, @@ -311,7 +477,8 @@ class ClickHouseSparkCatalog case _: NoSuchDatabaseException => // this is fine case _: NoSuchTableException => // this is fine } - BestEffortStagedTable(ident, super.createTable(ident, schema, partitions, properties), this) + val table = createCatalogTable(ident, schema, partitions, properties) + BestEffortStagedTable(ident, table, this) } } @@ -321,13 +488,22 @@ class ClickHouseSparkCatalog partitions: Array[Transform], properties: util.Map[String, String]): StagedTable = recordFrameProfile("DeltaCatalog", "stageCreate") { - if (CHDataSourceUtils.isClickHouseDataSourceName(getProvider(properties))) { + if ( + CHDataSourceUtils.isClickHouseDataSourceName(getProvider(properties)) || + DeltaSourceUtils.isDeltaDataSourceName(getProvider(properties)) + ) { new StagedDeltaTableV2(ident, schema, partitions, properties, TableCreationModes.Create) } else { - BestEffortStagedTable(ident, super.createTable(ident, schema, partitions, properties), this) + val table = createCatalogTable(ident, schema, partitions, properties) + BestEffortStagedTable(ident, table, this) } } + /** + * A staged delta table, which creates a HiveMetaStore entry and appends data if this was a + * CTAS/RTAS command. We have a ugly way of using this API right now, but it's the best way to + * maintain old behavior compatibility between Databricks Runtime and OSS Delta Lake. + */ private class StagedDeltaTableV2( ident: Identifier, override val schema: StructType, @@ -374,14 +550,18 @@ class ClickHouseSparkCatalog } } } - createClickHouseTable( - ident, - schema, - partitions, - props, - writeOptions, - asSelectQuery, - operation) + if (CHDataSourceUtils.isClickHouseDataSourceName(getProvider(properties))) { + createClickHouseTable( + ident, + schema, + partitions, + props, + writeOptions, + asSelectQuery, + operation) + } else { + createDeltaTable(ident, schema, partitions, props, writeOptions, asSelectQuery, operation) + } } override def name(): String = ident.name() @@ -454,20 +634,29 @@ trait SupportsPathIdentifier extends TableCatalog { protected def isPathIdentifier(ident: Identifier): Boolean = { // Should be a simple check of a special PathIdentifier class in the future try { - supportSQLOnFile && hasClickHouseNamespace(ident) && new Path(ident.name()).isAbsolute + supportSQLOnFile && (hasClickHouseNamespace(ident) || hasDeltaNamespace(ident)) && + new Path(ident.name()).isAbsolute } catch { case _: IllegalArgumentException => false } } + protected def isPathIdentifier(table: CatalogTable): Boolean = { + isPathIdentifier(table.identifier) + } + + protected def isPathIdentifier(tableIdentifier: TableIdentifier): Boolean = { + isPathIdentifier(Identifier.of(tableIdentifier.database.toArray, tableIdentifier.table)) + } + private def supportSQLOnFile: Boolean = spark.sessionState.conf.runSQLonFile - private def hasClickHouseNamespace(ident: Identifier): Boolean = { + protected def hasClickHouseNamespace(ident: Identifier): Boolean = { ident.namespace().length == 1 && CHDataSourceUtils.isClickHouseDataSourceName(ident.namespace().head) } - protected def isPathIdentifier(table: CatalogTable): Boolean = { - isPathIdentifier(Identifier.of(table.identifier.database.toArray, table.identifier.table)) + protected def hasDeltaNamespace(ident: Identifier): Boolean = { + ident.namespace().length == 1 && DeltaSourceUtils.isDeltaDataSourceName(ident.namespace().head) } } diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala new file mode 100644 index 000000000000..a097fc6cd4ab --- /dev/null +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseDeltaParquetWriteSuite.scala @@ -0,0 +1,1430 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.gluten.execution + +import org.apache.gluten.GlutenConfig + +import org.apache.spark.SparkConf +import org.apache.spark.sql.SaveMode +import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.files.TahoeFileIndex +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper + +import io.delta.tables.DeltaTable + +import java.io.File + +// Some sqls' line length exceeds 100 +// scalastyle:off line.size.limit + +class GlutenClickHouseDeltaParquetWriteSuite + extends GlutenClickHouseTPCHAbstractSuite + with AdaptiveSparkPlanHelper { + + override protected val needCopyParquetToTablePath = true + + override protected val tablesPath: String = basePath + "/tpch-data" + override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch" + override protected val queriesResults: String = rootPath + "mergetree-queries-output" + + /** Run Gluten + ClickHouse Backend with SortShuffleManager */ + override protected def sparkConf: SparkConf = { + super.sparkConf + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.io.compression.codec", "LZ4") + .set("spark.sql.shuffle.partitions", "5") + .set("spark.sql.autoBroadcastJoinThreshold", "10MB") + .set("spark.sql.adaptive.enabled", "true") + .set("spark.sql.files.maxPartitionBytes", "20000000") + .set("spark.gluten.sql.native.writer.enabled", "true") + .set("spark.sql.storeAssignmentPolicy", "legacy") + .set( + "spark.gluten.sql.columnar.backend.ch.runtime_settings.mergetree.merge_after_insert", + "false") + .set( + "spark.databricks.delta.retentionDurationCheck.enabled", + "false" + ) + } + + override protected def createTPCHNotNullTables(): Unit = { + createNotNullTPCHTablesInParquet(tablesPath) + } + + test("test parquet table write with the delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |LOCATION '$basePath/lineitem_delta_parquet' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_delta_parquet + | select /*+ REPARTITION(5) */ * from lineitem + |""".stripMargin) + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | lineitem_delta_parquet + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runTPCHQueryBySQL(1, sqlStr) { + df => + val plans = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + case w: WholeStageTransformer => w + } + assert(plans.size == 4) + + val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] + assert(parquetScan.nodeName.startsWith("Scan parquet ")) + + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + assert(addFiles.size == 5) + } + } + + test("test parquet insert overwrite with the delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_delta_parquet_insertoverwrite + | select * from lineitem + |""".stripMargin) + + spark.sql(s""" + | insert overwrite table lineitem_delta_parquet_insertoverwrite + | select * from lineitem where mod(l_orderkey,2) = 1 + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_delta_parquet_insertoverwrite + |""".stripMargin + assert( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) == 300001 + ) + } + + test("test parquet insert overwrite partitioned table with small table, static with delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite2; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite2 + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |PARTITIONED BY (l_shipdate) + |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite2' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_delta_parquet_insertoverwrite2 + | select * from lineitem + |""".stripMargin) + + spark.sql( + s""" + | insert overwrite table lineitem_delta_parquet_insertoverwrite2 + | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_delta_parquet_insertoverwrite2 + | + |""".stripMargin + assert( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) == 2418 + ) + } + + test("test parquet insert overwrite partitioned table with small table, dynamic with delta") { + withSQLConf(("spark.sql.sources.partitionOverwriteMode", "dynamic")) { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_insertoverwrite3 PURGE; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_insertoverwrite3 + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |PARTITIONED BY (l_shipdate) + |LOCATION '$basePath/lineitem_delta_parquet_insertoverwrite3' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_delta_parquet_insertoverwrite3 + | select * from lineitem + |""".stripMargin) + + spark.sql( + s""" + | insert overwrite table lineitem_delta_parquet_insertoverwrite3 + | select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + val sql2 = + s""" + | select count(*) from lineitem_delta_parquet_insertoverwrite3 + | + |""".stripMargin + assert( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) == 600572 + ) + } + } + + test("test parquet table update with the delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_update; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_update + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |LOCATION '$basePath/lineitem_delta_parquet_update' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_delta_parquet_update + | select /*+ REPARTITION(6) */ * from lineitem + |""".stripMargin) + + spark.sql( + s""" + | update lineitem_delta_parquet_update set l_returnflag = 'Z' where l_orderkey = 12647 + |""".stripMargin) + + { + val sql1 = + s""" + | select count(*) from lineitem_delta_parquet_update where l_returnflag = 'Z' + | + |""".stripMargin + + val df = spark.sql(sql1) + val result = df.collect() + assert( + // in test data, there are only 1 row with l_orderkey = 12647 + result.apply(0).get(0) == 1 + ) + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assert(scanExec.size == 1) + + val parquetScan = scanExec.head + assert(parquetScan.nodeName.startsWith("Scan parquet")) + + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + assert(addFiles.size == 4) + } + + val sql2 = + s""" + | select count(*) from lineitem_delta_parquet_update + | + |""".stripMargin + assert( + // total rows should remain unchanged + spark.sql(sql2).collect().apply(0).get(0) == 600572 + ) + } + + test("test parquet table delete with the delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_delete; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_delete + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |LOCATION '$basePath/lineitem_delta_parquet_delete' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_delta_parquet_delete + | select /*+ REPARTITION(6) */ * from lineitem + |""".stripMargin) + val df1 = spark.sql(s""" + | delete from lineitem_delta_parquet_delete where l_orderkey = 12647 + |""".stripMargin) + + { + val df = spark.sql(s""" + | select sum(l_linenumber) from lineitem_delta_parquet_delete + |""".stripMargin) + val result = df.collect() + assert( + result.apply(0).get(0) == 1802445 + ) + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + val parquetScan = scanExec.head + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + assert(addFiles.size == 4) + } + + { + spark.sql(s""" + | delete from lineitem_delta_parquet_delete where mod(l_orderkey, 3) = 2 + |""".stripMargin) + val df3 = spark.sql(s""" + | select sum(l_linenumber) from lineitem_delta_parquet_delete + |""".stripMargin) + assert( + df3.collect().apply(0).get(0) == 1200671 + ) + } + } + + test("test parquet table upsert with the delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_upsert; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_upsert + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |LOCATION '$basePath/lineitem_delta_parquet_upsert' + |""".stripMargin) + + spark.sql(s""" + | insert into table lineitem_delta_parquet_upsert + | select * from lineitem + |""".stripMargin) + + { + val df0 = spark.sql(s""" + | select sum(l_linenumber) from lineitem_delta_parquet_upsert + |""".stripMargin) + assert( + df0.collect().apply(0).get(0) == 1802446 + ) + } + + upsertSourceTableAndCheck("lineitem_delta_parquet_upsert") + } + + private def upsertSourceTableAndCheck(tableName: String) = { + // Why selecting l_orderkey having count(*) =1 ? + // Answer: to avoid "org.apache.spark.sql.delta.DeltaUnsupportedOperationException: + // Cannot perform Merge as multiple source rows matched and attempted to modify the same + // target row in the Delta table in possibly conflicting ways." + spark.sql(s""" + merge into $tableName + using ( + + select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, + 'Z' as `l_returnflag`, + l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment + from lineitem where l_orderkey in (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) and l_orderkey < 100000 + + union + + select l_orderkey + 10000000, + l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, + l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment + from lineitem where l_orderkey in (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) and l_orderkey < 100000 + + ) as updates + on updates.l_orderkey = $tableName.l_orderkey + when matched then update set * + when not matched then insert * + """.stripMargin) + + { + val df1 = spark.sql(s""" + | select count(*) from $tableName + |""".stripMargin) + assert( + df1.collect().apply(0).get(0) == 600572 + 3506 + ) + } + { + val df2 = + spark.sql(s""" + | select count(*) from $tableName where l_returnflag = 'Z' + |""".stripMargin) + assert( + df2.collect().apply(0).get(0) == 3506 + ) + } + + { + val df3 = + spark.sql(s""" + | select count(*) from $tableName where l_orderkey > 10000000 + |""".stripMargin) + assert( + df3.collect().apply(0).get(0) == 3506 + ) + } + } + + test("test parquet write with partition + delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_partition; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_partition + |( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |PARTITIONED BY (l_shipdate, l_returnflag) + |LOCATION '$basePath/lineitem_delta_parquet_partition' + |""".stripMargin) + + // dynamic partitions + spark.sql(s""" + | insert into table lineitem_delta_parquet_partition + | select * from lineitem + |""".stripMargin) + + // write with dataframe api + val source = spark.sql(s""" + |select + | l_orderkey , + | l_partkey , + | l_suppkey , + | l_linenumber , + | l_quantity , + | l_extendedprice , + | l_discount , + | l_tax , + | l_returnflag , + | l_linestatus , + | l_shipdate , + | l_commitdate , + | l_receiptdate , + | l_shipinstruct , + | l_shipmode , + | l_comment + | from lineitem + | where l_shipdate BETWEEN date'1993-01-01' AND date'1993-01-10' + |""".stripMargin) + + source.write + .format("delta") + .mode(SaveMode.Append) + .insertInto("lineitem_delta_parquet_partition") + + // static partition + spark.sql( + s""" + | insert into lineitem_delta_parquet_partition PARTITION (l_shipdate=date'1995-01-21', + | l_returnflag = 'A') + | (l_orderkey, + | l_partkey, + | l_suppkey, + | l_linenumber, + | l_quantity, + | l_extendedprice, + | l_discount, + | l_tax, + | l_linestatus, + | l_commitdate, + | l_receiptdate, + | l_shipinstruct, + | l_shipmode, + | l_comment) + | select l_orderkey, + | l_partkey, + | l_suppkey, + | l_linenumber, + | l_quantity, + | l_extendedprice, + | l_discount, + | l_tax, + | l_linestatus, + | l_commitdate, + | l_receiptdate, + | l_shipinstruct, + | l_shipmode, + | l_comment from lineitem + | where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | lineitem_delta_parquet_partition + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + df => + val result = df.collect() + assert(result.size == 4) + assert(result(0).getString(0).equals("A")) + assert(result(0).getString(1).equals("F")) + assert(result(0).getDouble(2) == 3865234.0) + + assert(result(2).getString(0).equals("N")) + assert(result(2).getString(1).equals("O")) + assert(result(2).getDouble(2) == 7454519.0) + + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assert(scanExec.size == 1) + + val parquetScan = scanExec(0) + assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.metrics("numFiles").value == 3745) + + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + + assert(addFiles.size == 3836) + assert( + addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1992-06-01")).size == 2) + assert( + addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1993-01-01")).size == 4) + assert( + addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1995-01-21")).size == 3) + } + } + + test("test parquet CTAS simple with the delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_ctas1; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE lineitem_delta_parquet_ctas1 + |USING delta + |LOCATION '$basePath/lineitem_delta_parquet_ctas1' + | as select * from lineitem + |""".stripMargin) + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | lineitem_delta_parquet_ctas1 + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runTPCHQueryBySQL(1, sqlStr) { + df => + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assert(scanExec.size == 1) + + val parquetScan = scanExec(0) + assert(parquetScan.nodeName.startsWith("Scan parquet")) + + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + assert(addFiles.size == 1) + } + } + + test("test parquet CTAS complex with the delta") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_ctas2; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_ctas2 + |USING delta + |PARTITIONED BY (l_shipdate) + |LOCATION '$basePath/lineitem_mergetree_ctas2' + | as select * from lineitem + |""".stripMargin) + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | lineitem_delta_parquet_ctas2 + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runTPCHQueryBySQL(1, sqlStr) { _ => {} } + + } + + test("test path based parquet write with the delta") { + val dataPath = s"$basePath/lineitem_delta_parquet_filebased" + clearDataPath(dataPath) + + val sourceDF = spark.sql(s""" + |select * from lineitem + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE delta.`$dataPath` ( + | l_orderkey bigint, + | l_partkey bigint, + | l_suppkey bigint, + | l_linenumber bigint, + | l_quantity double, + | l_extendedprice double, + | l_discount double, + | l_tax double, + | l_returnflag string, + | l_linestatus string, + | l_shipdate date, + | l_commitdate date, + | l_receiptdate date, + | l_shipinstruct string, + | l_shipmode string, + | l_comment string + |) + |USING delta + |""".stripMargin) + + sourceDF.write + .format("delta") + .mode(SaveMode.Append) + .save(dataPath) + sourceDF.write + .format("delta") + .mode(SaveMode.Append) + .save(dataPath) + sourceDF.write + .format("delta") + .mode(SaveMode.Overwrite) + .save(dataPath) + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | delta.`$dataPath` + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runTPCHQueryBySQL(1, sqlStr) { + df => + val plans = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + case w: WholeStageTransformer => w + } + assert(plans.size == 4) + + val parquetScan = plans(3).asInstanceOf[FileSourceScanExecTransformer] + assert(parquetScan.nodeName.startsWith("Scan parquet")) + + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + assert(addFiles.size == 1) + } + + val result = spark.read + .format("delta") + .load(dataPath) + .where("l_shipdate = date'1998-09-02'") + .count() + assert(result == 183) + } + + test( + "test path based parquet insert overwrite partitioned table with small table, static with delta") { + val dataPath = s"$basePath/lineitem_delta_parquet_insertoverwrite2" + clearDataPath(dataPath) + + val sourceDF = spark.sql(s""" + |select * from lineitem + |""".stripMargin) + + sourceDF.write + .format("delta") + .mode(SaveMode.Append) + .partitionBy("l_shipdate") + .save(dataPath) + + val sourceDF1 = spark.sql( + s""" + |select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + sourceDF1.write + .format("delta") + .mode(SaveMode.Overwrite) + .partitionBy("l_shipdate") + .save(dataPath) + + val result = spark.read + .format("delta") + .load(dataPath) + .count() + assert(result == 2418) + } + + test( + "test path based parquet insert overwrite partitioned table with small table, dynamic with delta") { + withSQLConf(("spark.sql.sources.partitionOverwriteMode", "dynamic")) { + val dataPath = s"$basePath/lineitem_delta_parquet_insertoverwrite3" + clearDataPath(dataPath) + + val sourceDF = spark.sql(s""" + |select * from lineitem + |""".stripMargin) + + sourceDF.write + .format("delta") + .mode(SaveMode.Append) + .partitionBy("l_shipdate") + .save(dataPath) + + val sourceDF1 = spark.sql( + s""" + |select * from lineitem where l_shipdate BETWEEN date'1993-02-01' AND date'1993-02-10' + |""".stripMargin) + sourceDF1.write + .format("delta") + .mode(SaveMode.Overwrite) + .partitionBy("l_shipdate") + .save(dataPath) + + val result = spark.read + .format("delta") + .load(dataPath) + .count() + assert(result == 600572) + } + } + + test("test path based parquet update with the delta") { + val dataPath = s"$basePath/lineitem_delta_parquet_update" + clearDataPath(dataPath) + + val sourceDF = spark.sql(s""" + |select /*+ REPARTITION(6) */ * from lineitem + |""".stripMargin) + + sourceDF.write + .format("delta") + .mode(SaveMode.Append) + .save(dataPath) + + spark.sql(s""" + | update delta.`$dataPath` set l_returnflag = 'Z' where l_orderkey = 12647 + |""".stripMargin) + + { + val df = spark.read + .format("delta") + .load(dataPath) + .where("l_returnflag = 'Z'") + assert(df.count() == 1) + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assert(scanExec.size == 1) + + val parquetScan = scanExec.head + assert(parquetScan.nodeName.startsWith("Scan parquet")) + + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + assert(addFiles.size == 4) + } + + val clickhouseTable = DeltaTable.forPath(spark, dataPath) + clickhouseTable.updateExpr("l_orderkey = 10086", Map("l_returnflag" -> "'X'")) + + { + val df = spark.read + .format("delta") + .load(dataPath) + .where("l_returnflag = 'X'") + assert(df.count() == 1) + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assert(scanExec.size == 1) + + val parquetScan = scanExec.head + assert(parquetScan.nodeName.startsWith("Scan parquet")) + + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + assert(addFiles.size == 3) + } + + val df = spark.read + .format("delta") + .load(dataPath) + assert(df.count() == 600572) + } + + test("test path based parquet delete with the delta") { + val dataPath = s"$basePath/lineitem_delta_parquet_delete" + clearDataPath(dataPath) + + val sourceDF = spark.sql(s""" + |select /*+ REPARTITION(6) */ * from lineitem + |""".stripMargin) + + sourceDF.write + .format("delta") + .mode(SaveMode.Append) + .save(dataPath) + + spark.sql(s""" + | delete from delta.`$dataPath` where l_orderkey = 12647 + |""".stripMargin) + val df = spark.read + .format("delta") + .load(dataPath) + assert(df.count() == 600571) + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + val parquetScan = scanExec.head + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + assert(addFiles.size == 4) + + val clickhouseTable = DeltaTable.forPath(spark, dataPath) + clickhouseTable.delete("mod(l_orderkey, 3) = 2") + val df1 = spark.read + .format("delta") + .load(dataPath) + assert(df1.count() == 400089) + } + + test("test path based parquet upsert with the delta") { + val dataPath = s"$basePath/lineitem_delta_parquet_upsert" + clearDataPath(dataPath) + + val sourceDF = spark.sql(s""" + |select /*+ REPARTITION(6) */ * from lineitem + |""".stripMargin) + + sourceDF.write + .format("delta") + .mode(SaveMode.Append) + .save(dataPath) + + val df0 = spark.sql(s""" + | select count(*) from delta.`$dataPath` + |""".stripMargin) + assert( + df0.collect().apply(0).get(0) == 600572 + ) + upsertPathBasedSourceTableAndCheck(dataPath) + } + + private def upsertPathBasedSourceTableAndCheck(dataPath: String) = { + // Why selecting l_orderkey having count(*) =1 ? + // Answer: to avoid "org.apache.spark.sql.delta.DeltaUnsupportedOperationException: + // Cannot perform Merge as multiple source rows matched and attempted to modify the same + // target row in the Delta table in possibly conflicting ways." + spark.sql(s""" + merge into delta.`$dataPath` + using ( + + select l_orderkey, l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, + 'Z' as `l_returnflag`, + l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment + from lineitem where l_orderkey in (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) and l_orderkey < 100000 + + union + + select l_orderkey + 10000000, + l_partkey, l_suppkey, l_linenumber, l_quantity, l_extendedprice, l_discount, l_tax, l_returnflag, + l_linestatus, l_shipdate, l_commitdate, l_receiptdate, l_shipinstruct, l_shipmode, l_comment + from lineitem where l_orderkey in (select l_orderkey from lineitem group by l_orderkey having count(*) =1 ) and l_orderkey < 100000 + + ) as updates + on updates.l_orderkey = delta.`$dataPath`.l_orderkey + when matched then update set * + when not matched then insert * + """.stripMargin) + + { + val df1 = spark.sql(s""" + | select count(*) from delta.`$dataPath` + |""".stripMargin) + assert( + df1.collect().apply(0).get(0) == 600572 + 3506 + ) + } + { + val df2 = + spark.sql(s""" + | select count(*) from delta.`$dataPath` where l_returnflag = 'Z' + |""".stripMargin) + assert( + df2.collect().apply(0).get(0) == 3506 + ) + } + + { + val df3 = + spark.sql(s""" + | select count(*) from delta.`$dataPath` where l_orderkey > 10000000 + |""".stripMargin) + assert( + df3.collect().apply(0).get(0) == 3506 + ) + } + } + + test("test path based parquet write with partition + delta") { + val dataPath = s"$basePath/lineitem_delta_parquet_partition" + clearDataPath(dataPath) + + val sourceDF = spark.sql(s""" + |select * from lineitem + |""".stripMargin) + + sourceDF.write + .format("delta") + .partitionBy("l_shipdate", "l_returnflag") + .mode(SaveMode.Append) + .save(dataPath) + + val sourceDF1 = spark.sql( + s""" + |select * from lineitem where l_shipdate BETWEEN date'1993-01-01' AND date'1993-01-10' + |""".stripMargin) + + sourceDF1.write + .format("delta") + .mode(SaveMode.Append) + .save(dataPath) + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | delta.`$dataPath` + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runTPCHQueryBySQL(1, sqlStr, compareResult = false) { + df => + val result = df.collect() + assert(result.size == 4) + assert(result(0).getString(0).equals("A")) + assert(result(0).getString(1).equals("F")) + assert(result(0).getDouble(2) == 3803858.0) + + assert(result(2).getString(0).equals("N")) + assert(result(2).getString(1).equals("O")) + assert(result(2).getDouble(2) == 7454519.0) + + val scanExec = collect(df.queryExecution.executedPlan) { + case f: FileSourceScanExecTransformer => f + } + assert(scanExec.size == 1) + + val parquetScan = scanExec(0) + assert(parquetScan.nodeName.startsWith("Scan parquet")) + assert(parquetScan.metrics("numFiles").value == 3744) + + val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex] + val addFiles = fileIndex.matchingFiles(Nil, Nil).map(f => f.asInstanceOf[AddFile]) + + assert(addFiles.size == 3835) + assert( + addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1992-06-01")).size == 2) + assert( + addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1993-01-01")).size == 4) + assert( + addFiles.filter(_.partitionValues.get("l_shipdate").get.equals("1995-01-21")).size == 2) + } + } + + test("test path based parquet CTAS with delta") { + val dataPath = s"$basePath/lineitem_delta_parquet_ctas" + clearDataPath(dataPath) + + spark.sql(s""" + |CREATE TABLE delta.`$dataPath` + |USING delta + |PARTITIONED BY (l_shipdate) + | as select * from lineitem + |""".stripMargin) + + val sqlStr = + s""" + |SELECT + | l_returnflag, + | l_linestatus, + | sum(l_quantity) AS sum_qty, + | sum(l_extendedprice) AS sum_base_price, + | sum(l_extendedprice * (1 - l_discount)) AS sum_disc_price, + | sum(l_extendedprice * (1 - l_discount) * (1 + l_tax)) AS sum_charge, + | avg(l_quantity) AS avg_qty, + | avg(l_extendedprice) AS avg_price, + | avg(l_discount) AS avg_disc, + | count(*) AS count_order + |FROM + | delta.`$dataPath` + |WHERE + | l_shipdate <= date'1998-09-02' - interval 1 day + |GROUP BY + | l_returnflag, + | l_linestatus + |ORDER BY + | l_returnflag, + | l_linestatus; + | + |""".stripMargin + runTPCHQueryBySQL(1, sqlStr) { _ => {} } + } + + test("test parquet optimize basic") { + withSQLConf("spark.databricks.delta.optimize.maxFileSize" -> "20000000") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_optimize + |USING delta + |LOCATION '$basePath/lineitem_delta_parquet_optimize' + | as select /*+ REPARTITION(20) */ * from lineitem + |""".stripMargin) + + spark.sql("optimize lineitem_delta_parquet_optimize") + val ret = spark.sql("select count(*) from lineitem_delta_parquet_optimize").collect() + assert(ret.apply(0).get(0) == 600572) + + assert( + countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize")) == 24 + ) + } + } + + def countFiles(directory: File): Int = { + if (directory.exists && directory.isDirectory) { + val files = directory.listFiles + val count = files + .filter(!_.getName.endsWith(".crc")) + .count(_.isFile) + files.filter(_.isDirectory).map(countFiles).sum + count + } else { + 0 + } + } + + test("test parquet optimize partitioned by one low card column") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize_p2; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_optimize_p2 + |USING delta + |PARTITIONED BY (l_returnflag) + |LOCATION '$basePath/lineitem_delta_parquet_optimize_p2' + | as select /*+ REPARTITION(6) */ * from lineitem + |""".stripMargin) + + spark.sparkContext.setJobGroup("test3", "test3") + spark.sql("optimize lineitem_delta_parquet_optimize_p2") + val job_ids = spark.sparkContext.statusTracker.getJobIdsForGroup("test3") + if (sparkVersion.equals("3.2")) { + assert(job_ids.size == 7) // WILL trigger actual merge job + } else { + assert(job_ids.size == 8) // WILL trigger actual merge job + } + + spark.sparkContext.clearJobGroup() + + val ret = spark.sql("select count(*) from lineitem_delta_parquet_optimize_p2").collect() + assert(ret.apply(0).get(0) == 600572) + + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) == 23) + spark.sql("VACUUM lineitem_delta_parquet_optimize_p2 RETAIN 0 HOURS") + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) == 5) + } else { + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p2")) == 7) + } + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") + + val ret2 = spark.sql("select count(*) from lineitem_delta_parquet_optimize_p2").collect() + assert(ret2.apply(0).get(0) == 600572) + } + + test("test parquet optimize parallel delete") { + withSQLConf("spark.databricks.delta.vacuum.parallelDelete.enabled" -> "true") { + spark.sql(s""" + |DROP TABLE IF EXISTS lineitem_delta_parquet_optimize_p4; + |""".stripMargin) + + spark.sql(s""" + |CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_optimize_p4 + |USING delta + |PARTITIONED BY (l_linenumber,l_returnflag) + |LOCATION '$basePath/lineitem_delta_parquet_optimize_p4' + | as select /*+ REPARTITION(6) */ * from lineitem + |""".stripMargin) + + spark.sql("optimize lineitem_delta_parquet_optimize_p4") + val ret = spark.sql("select count(*) from lineitem_delta_parquet_optimize_p4").collect() + assert(ret.apply(0).get(0) == 600572) + + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) == 149) + spark.sql("VACUUM lineitem_delta_parquet_optimize_p4 RETAIN 0 HOURS") + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) == 23) + } else { + assert(countFiles(new File(s"$basePath/lineitem_delta_parquet_optimize_p4")) == 25) + } + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") + + val ret2 = spark.sql("select count(*) from lineitem_delta_parquet_optimize_p4").collect() + assert(ret2.apply(0).get(0) == 600572) + } + } + + test("test parquet optimize with the path based table") { + val dataPath = s"$basePath/lineitem_delta_parquet_optimize_path_based" + clearDataPath(dataPath) + withSQLConf( + ("spark.databricks.delta.optimize.maxFileSize" -> "1000000"), + ("spark.databricks.delta.optimize.minFileSize" -> "838000")) { + + val sourceDF = spark.sql(s""" + |select /*+ REPARTITION(50) */ * from lineitem + |""".stripMargin) + + sourceDF.write + .format("delta") + .mode(SaveMode.Append) + .save(dataPath) + + val clickhouseTable = DeltaTable.forPath(spark, dataPath) + clickhouseTable.optimize().executeCompaction() + + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") + clickhouseTable.vacuum(0.0) + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(dataPath)) == 27) + } else { + assert(countFiles(new File(dataPath)) == 29) + } + + val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() + assert(ret.apply(0).get(0) == 600572) + } + + withSQLConf( + ("spark.databricks.delta.optimize.maxFileSize" -> "10000000"), + ("spark.databricks.delta.optimize.minFileSize" -> "1000000")) { + + val clickhouseTable = DeltaTable.forPath(spark, dataPath) + clickhouseTable.optimize().executeCompaction() + + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") + clickhouseTable.vacuum(0.0) + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(dataPath)) == 6) + } else { + assert(countFiles(new File(dataPath)) == 12) + } + + val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() + assert(ret.apply(0).get(0) == 600572) + } + + // now merge all parts (testing merging from merged parts) + val clickhouseTable = DeltaTable.forPath(spark, dataPath) + clickhouseTable.optimize().executeCompaction() + + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") + clickhouseTable.vacuum(0.0) + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(dataPath)) == 5) + } else { + assert(countFiles(new File(dataPath)) == 13) + } + + val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() + assert(ret.apply(0).get(0) == 600572) + } +} +// scalastyle:off line.size.limit diff --git a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeOptimizeSuite.scala b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeOptimizeSuite.scala index f016f9dc5d14..c94a3bf50c63 100644 --- a/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeOptimizeSuite.scala +++ b/backends-clickhouse/src/test/scala/org/apache/gluten/execution/GlutenClickHouseMergeTreeOptimizeSuite.scala @@ -127,7 +127,13 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p")) == 22728) spark.sql("VACUUM lineitem_mergetree_optimize_p RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p")) == 22728) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p")) == 22728) + } else { + // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p")) == 22730) + } + spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p").collect() @@ -150,7 +156,12 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sparkContext.setJobGroup("test2", "test2") spark.sql("optimize lineitem_mergetree_optimize_p2") val job_ids = spark.sparkContext.statusTracker.getJobIdsForGroup("test2") - assert(job_ids.size == 7) // WILL trigger actual merge job + if (sparkVersion.equals("3.2")) { + assert(job_ids.size == 7) // WILL trigger actual merge job + } else { + assert(job_ids.size == 8) // WILL trigger actual merge job + } + spark.sparkContext.clearJobGroup() val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p2").collect() @@ -159,10 +170,18 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2")) == 372) spark.sql("VACUUM lineitem_mergetree_optimize_p2 RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2")) == 239) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2")) == 239) + } else { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2")) == 241) + } spark.sql("VACUUM lineitem_mergetree_optimize_p2 RETAIN 0 HOURS") // the second VACUUM will remove some empty folders - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2")) == 220) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2")) == 220) + } else { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p2")) == 226) + } spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p2").collect() @@ -190,9 +209,17 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3")) == 516) spark.sql("VACUUM lineitem_mergetree_optimize_p3 RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3")) == 306) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3")) == 306) + } else { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3")) == 308) + } spark.sql("VACUUM lineitem_mergetree_optimize_p3 RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3")) == 276) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3")) == 276) + } else { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p3")) == 282) + } spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p3").collect() @@ -221,9 +248,17 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4")) == 516) spark.sql("VACUUM lineitem_mergetree_optimize_p4 RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4")) == 306) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4")) == 306) + } else { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4")) == 308) + } spark.sql("VACUUM lineitem_mergetree_optimize_p4 RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4")) == 276) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4")) == 276) + } else { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p4")) == 282) + } spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") val ret2 = spark.sql("select count(*) from lineitem_mergetree_optimize_p4").collect() @@ -251,7 +286,13 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 99) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 99) + } else { + // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. + // this case will create a checkpoint + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 105) + } spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p5").collect() @@ -271,7 +312,12 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 93) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 93) + } else { + // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 104) + } spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p5").collect() @@ -284,7 +330,12 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=false") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") spark.sql("VACUUM lineitem_mergetree_optimize_p5 RETAIN 0 HOURS") - assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 77) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 77) + } else { + // For Spark 3.3 + Delta 2.3, vacuum command will create two commit files in deltalog dir. + assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p5")) == 93) + } spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") val ret = spark.sql("select count(*) from lineitem_mergetree_optimize_p5").collect() @@ -318,7 +369,7 @@ class GlutenClickHouseMergeTreeOptimizeSuite spark.sql("VACUUM lineitem_mergetree_optimize_p6 RETAIN 0 HOURS") spark.sql("VACUUM lineitem_mergetree_optimize_p6 RETAIN 0 HOURS") assert(countFiles(new File(s"$basePath/lineitem_mergetree_optimize_p6")) == { - if (sparkVersion.equals("3.2")) 315 else 321 + if (sparkVersion.equals("3.2")) 315 else 327 }) spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") @@ -393,7 +444,11 @@ class GlutenClickHouseMergeTreeOptimizeSuite clickhouseTable.vacuum(0.0) clickhouseTable.vacuum(0.0) spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") - assert(countFiles(new File(dataPath)) == 99) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(dataPath)) == 99) + } else { + assert(countFiles(new File(dataPath)) == 105) + } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() assert(ret.apply(0).get(0) == 600572) @@ -414,7 +469,11 @@ class GlutenClickHouseMergeTreeOptimizeSuite clickhouseTable.vacuum(0.0) clickhouseTable.vacuum(0.0) spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") - assert(countFiles(new File(dataPath)) == 93) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(dataPath)) == 93) + } else { + assert(countFiles(new File(dataPath)) == 104) + } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() assert(ret.apply(0).get(0) == 600572) @@ -428,7 +487,11 @@ class GlutenClickHouseMergeTreeOptimizeSuite clickhouseTable.vacuum(0.0) clickhouseTable.vacuum(0.0) spark.sql(s"set ${GlutenConfig.GLUTEN_ENABLED.key}=true") - assert(countFiles(new File(dataPath)) == 77) + if (sparkVersion.equals("3.2")) { + assert(countFiles(new File(dataPath)) == 77) + } else { + assert(countFiles(new File(dataPath)) == 93) + } val ret = spark.sql(s"select count(*) from clickhouse.`$dataPath`").collect() assert(ret.apply(0).get(0) == 600572) diff --git a/gluten-core/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala b/gluten-core/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala index ff905251b8ae..f4855840256b 100644 --- a/gluten-core/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala +++ b/gluten-core/src/main/scala/org/apache/gluten/execution/FileSourceScanExecTransformer.scala @@ -177,6 +177,7 @@ abstract class FileSourceScanExecTransformerBase( relation.fileFormat.getClass.getSimpleName match { case "OrcFileFormat" => ReadFileFormat.OrcReadFormat case "ParquetFileFormat" => ReadFileFormat.ParquetReadFormat + case "DeltaParquetFileFormat" => ReadFileFormat.ParquetReadFormat case "DwrfFileFormat" => ReadFileFormat.DwrfReadFormat case "DeltaMergeTreeFileFormat" => ReadFileFormat.MergeTreeReadFormat case "CSVFileFormat" => ReadFileFormat.TextReadFormat diff --git a/gluten-ut/pom.xml b/gluten-ut/pom.xml index 2396087fcc33..79afa94c8e32 100644 --- a/gluten-ut/pom.xml +++ b/gluten-ut/pom.xml @@ -92,7 +92,7 @@ io.delta ${delta.package.name}_${scala.binary.version} - provided + test diff --git a/pom.xml b/pom.xml index 0f37bcbf1851..63c53f109302 100644 --- a/pom.xml +++ b/pom.xml @@ -42,8 +42,6 @@ ${java.version} ${java.version} 2.9.3 - 2.0.1 - 20 2.12 2.12.15 3 @@ -155,8 +153,8 @@ 1.3.1 delta-core - 2.2.0 - 22 + 2.3.0 + 23 @@ -181,8 +179,8 @@ 3.5.1 1.5.0 delta-spark - 3.1.0 - 31 + 3.2.0 + 32 2.15.1 3.3.4