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