From c27f35aa6200441f446b138581a3cd2d0b72616b Mon Sep 17 00:00:00 2001 From: Mohammad Linjawi Date: Sun, 31 May 2026 12:06:43 +0300 Subject: [PATCH 01/16] [VL][Delta] Add DV scan info extraction utility --- .../delta/DeltaDeletionVectorScanInfo.scala | 208 +++++++++++++++++ .../DeltaDeletionVectorScanInfoSuite.scala | 154 +++++++++++++ .../delta/DeltaDeletionVectorScanInfo.scala | 218 ++++++++++++++++++ .../DeltaDeletionVectorScanInfoSuite.scala | 154 +++++++++++++ 4 files changed, 734 insertions(+) create mode 100644 backends-velox/src-delta33/main/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfo.scala create mode 100644 backends-velox/src-delta33/test/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfoSuite.scala create mode 100644 backends-velox/src-delta40/main/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfo.scala create mode 100644 backends-velox/src-delta40/test/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfoSuite.scala diff --git a/backends-velox/src-delta33/main/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfo.scala b/backends-velox/src-delta33/main/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfo.scala new file mode 100644 index 00000000000..cad49ea2ffd --- /dev/null +++ b/backends-velox/src-delta33/main/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfo.scala @@ -0,0 +1,208 @@ +/* + * 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.delta + +import org.apache.gluten.sql.shims.SparkShimLoader + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.GlutenDeltaParquetFileFormat +import org.apache.spark.sql.delta.actions.DeletionVectorDescriptor +import org.apache.spark.sql.delta.deletionvectors.{RoaringBitmapArrayFormat, StoredBitmap} +import org.apache.spark.sql.delta.storage.dv.HadoopFileSystemDVStore +import org.apache.spark.sql.execution.datasources.PartitionedFile + +import org.apache.hadoop.fs.Path + +import java.util.{ArrayList => JArrayList} + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +object DeltaDeletionVectorScanInfo { + object RowIndexFilterType extends Enumeration { + type RowIndexFilterType = Value + val KEEP_ALL, IF_CONTAINED, IF_NOT_CONTAINED = Value + } + + import RowIndexFilterType._ + + final case class DeletionVectorInfo( + rowIndexFilterType: RowIndexFilterType, + descriptor: Option[DeletionVectorDescriptor], + serializedDeletionVector: Array[Byte]) { + def hasDeletionVector: Boolean = descriptor.isDefined + def cardinality: Long = descriptor.map(_.cardinality).getOrElse(0L) + } + + final case class PartitionFileScanInfo( + normalizedOtherMetadataColumns: Map[String, Object], + deletionVectorInfo: DeletionVectorInfo) + + private val RowIndexFilterIdEncoded = + GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_ID_ENCODED + private val RowIndexFilterTypeKey = + GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_TYPE + + def extract( + spark: SparkSession, + partitionColumnCount: Int, + file: PartitionedFile): PartitionFileScanInfo = { + val metadata = otherMetadataColumns(file) + val normalizedMetadata = metadata -- Seq(RowIndexFilterIdEncoded, RowIndexFilterTypeKey) + val dvInfo = extractDeletionVectorInfo(spark, partitionColumnCount, file, metadata) + PartitionFileScanInfo(normalizedMetadata, dvInfo) + } + + def extractAll( + spark: SparkSession, + partitionColumnCount: Int, + files: Seq[PartitionedFile]): Seq[PartitionFileScanInfo] = { + files.map(extract(spark, partitionColumnCount, _)) + } + + def extractAllFromJava( + spark: SparkSession, + partitionColumnCount: Int, + files: java.util.List[PartitionedFile]): java.util.List[PartitionFileScanInfo] = { + new JArrayList(extractAll(spark, partitionColumnCount, files.asScala.toSeq).asJava) + } + + private def extractDeletionVectorInfo( + spark: SparkSession, + partitionColumnCount: Int, + file: PartitionedFile, + metadata: Map[String, Object]): DeletionVectorInfo = { + val descriptorValue = metadata.get(RowIndexFilterIdEncoded) + val filterTypeValue = metadata.get(RowIndexFilterTypeKey) + + (descriptorValue, filterTypeValue) match { + case (None, None) => + DeletionVectorInfo(KEEP_ALL, None, Array.emptyByteArray) + case (Some(encodedDescriptor), Some(filterType)) => + val descriptor = parseDescriptor(encodedDescriptor.toString) + val serializedPayload = serializePayload(spark, partitionColumnCount, file, descriptor) + DeletionVectorInfo( + parseRowIndexFilterType(filterType.toString), + Some(descriptor), + serializedPayload) + case _ => + throw new IllegalStateException( + s"Both $RowIndexFilterIdEncoded and $RowIndexFilterTypeKey must either be present or absent") + } + } + + private def otherMetadataColumns(file: PartitionedFile): Map[String, Object] = { + val otherMetadata = + SparkShimLoader.getSparkShims.getOtherConstantMetadataColumnValues(file) + if (otherMetadata == null) { + Map.empty + } else { + otherMetadata.asScala.toMap + } + } + + private def parseDescriptor(encodedDescriptor: String): DeletionVectorDescriptor = { + try { + DeletionVectorDescriptor.deserializeFromBase64(encodedDescriptor) + } catch { + case NonFatal(e) => + throw new IllegalArgumentException("Unable to parse Delta deletion vector descriptor", e) + } + } + + private def parseRowIndexFilterType(filterType: String): RowIndexFilterType = { + filterType match { + case "IF_CONTAINED" => IF_CONTAINED + case "IF_NOT_CONTAINED" => IF_NOT_CONTAINED + case "KEEP_ALL" => KEEP_ALL + case unexpected => + throw new IllegalStateException(s"Unexpected row index filter type: $unexpected") + } + } + + private def serializePayload( + spark: SparkSession, + partitionColumnCount: Int, + file: PartitionedFile, + descriptor: DeletionVectorDescriptor): Array[Byte] = { + val tablePath = resolveTablePath(spark, partitionColumnCount, file) + if (tablePath == null) { + throw new IllegalStateException( + "Unable to resolve Delta table path while materializing deletion vector payload") + } + val dvStore = new HadoopFileSystemDVStore(spark.sessionState.newHadoopConf()) + StoredBitmap + .create(descriptor, tablePath) + .load(dvStore) + .serializeAsByteArray(RoaringBitmapArrayFormat.Portable) + } + + private def resolveTablePath( + spark: SparkSession, + partitionColumnCount: Int, + file: PartitionedFile): Path = { + val fileParent = new Path(unescapePathName(file.filePath.toString)).getParent + var tablePath = fileParent + for (_ <- 0 until partitionColumnCount) { + tablePath = tablePath.getParent + } + if (tablePath != null && isDeltaTablePath(spark, tablePath)) { + return tablePath + } + + var candidate = fileParent + while (candidate != null && !isDeltaTablePath(spark, candidate)) { + candidate = candidate.getParent + } + if (candidate != null) candidate else tablePath + } + + private def isDeltaTablePath(spark: SparkSession, tablePath: Path): Boolean = { + val deltaLogPath = new Path(tablePath, "_delta_log") + try { + deltaLogPath.getFileSystem(spark.sessionState.newHadoopConf()).exists(deltaLogPath) + } catch { + case NonFatal(_) => false + } + } + + private def unescapePathName(path: String): String = { + if (path == null || path.indexOf('%') < 0) { + path + } else { + val builder = new StringBuilder(path.length) + var index = 0 + while (index < path.length) { + if (path.charAt(index) == '%' && index + 2 < path.length) { + val high = Character.digit(path.charAt(index + 1), 16) + val low = Character.digit(path.charAt(index + 2), 16) + if (high >= 0 && low >= 0) { + builder.append(((high << 4) | low).toChar) + index += 3 + } else { + builder.append(path.charAt(index)) + index += 1 + } + } else { + builder.append(path.charAt(index)) + index += 1 + } + } + builder.toString() + } + } +} diff --git a/backends-velox/src-delta33/test/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfoSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfoSuite.scala new file mode 100644 index 00000000000..cbb70817ba4 --- /dev/null +++ b/backends-velox/src-delta33/test/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfoSuite.scala @@ -0,0 +1,154 @@ +/* + * 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.delta + +import org.apache.gluten.delta.DeltaDeletionVectorScanInfo.RowIndexFilterType + +import org.apache.spark.SparkConf +import org.apache.spark.paths.SparkPath +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.delta.{DeltaLog, GlutenDeltaParquetFileFormat} +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import org.apache.spark.sql.delta.test.DeltaSQLTestUtils +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.tags.ExtendedSQLTest + +import io.delta.sql.DeltaSparkSessionExtension +import org.apache.hadoop.fs.Path + +@ExtendedSQLTest +class DeltaDeletionVectorScanInfoSuite + extends QueryTest + with SharedSparkSession + with DeltaSQLTestUtils { + + import testImplicits._ + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, classOf[DeltaSparkSessionExtension].getName) + .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) + .set("spark.databricks.delta.snapshotPartitions", "2") + } + + test("extracts essential Delta DV scan info from split metadata") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)") + + val dataFile = DeltaLog + .forTable(spark, new Path(path)) + .update() + .allFiles + .collect() + .find(_.deletionVector != null) + .get + val partitionedFile = partitionedFileWithMetadata( + path, + dataFile.path, + dataFile.size, + Map( + GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_ID_ENCODED -> + dataFile.deletionVector.serializeToBase64(), + GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_TYPE -> "IF_CONTAINED", + "kept_key" -> "kept_value" + ) + ) + + val scanInfo = DeltaDeletionVectorScanInfo.extract(spark, 0, partitionedFile) + val dvInfo = scanInfo.deletionVectorInfo + + assert(dvInfo.hasDeletionVector) + assert(dvInfo.rowIndexFilterType == RowIndexFilterType.IF_CONTAINED) + assert(dvInfo.cardinality == dataFile.deletionVector.cardinality) + assert(dvInfo.serializedDeletionVector.nonEmpty) + assert(scanInfo.normalizedOtherMetadataColumns == Map("kept_key" -> "kept_value")) + } + } + + test("returns keep-all scan info when Delta DV metadata is absent") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a")).toDF("id", "value").coalesce(1).write.format("delta").save(path) + + val dataFile = DeltaLog.forTable(spark, new Path(path)).update().allFiles.collect().head + val partitionedFile = partitionedFileWithMetadata( + path, + dataFile.path, + dataFile.size, + Map("kept_key" -> "kept_value")) + + val scanInfo = DeltaDeletionVectorScanInfo.extract(spark, 0, partitionedFile) + val dvInfo = scanInfo.deletionVectorInfo + + assert(!dvInfo.hasDeletionVector) + assert(dvInfo.rowIndexFilterType == RowIndexFilterType.KEEP_ALL) + assert(dvInfo.cardinality == 0L) + assert(dvInfo.serializedDeletionVector.isEmpty) + assert(scanInfo.normalizedOtherMetadataColumns == Map("kept_key" -> "kept_value")) + } + } + + test("rejects partial Delta DV split metadata") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a")).toDF("id", "value").coalesce(1).write.format("delta").save(path) + + val dataFile = DeltaLog.forTable(spark, new Path(path)).update().allFiles.collect().head + val partitionedFile = partitionedFileWithMetadata( + path, + dataFile.path, + dataFile.size, + Map(GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_TYPE -> "IF_CONTAINED")) + + val error = intercept[IllegalStateException] { + DeltaDeletionVectorScanInfo.extract(spark, 0, partitionedFile) + } + assert(error.getMessage.contains("must either be present or absent")) + } + } + + private def partitionedFileWithMetadata( + tablePath: String, + relativeFilePath: String, + fileSize: Long, + metadata: Map[String, Object]): PartitionedFile = { + PartitionedFile( + partitionValues = InternalRow.empty, + filePath = SparkPath.fromPath(new Path(tablePath, relativeFilePath)), + start = 0L, + length = fileSize, + fileSize = fileSize, + otherConstantMetadataColumnValues = metadata + ) + } +} diff --git a/backends-velox/src-delta40/main/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfo.scala b/backends-velox/src-delta40/main/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfo.scala new file mode 100644 index 00000000000..12df2338784 --- /dev/null +++ b/backends-velox/src-delta40/main/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfo.scala @@ -0,0 +1,218 @@ +/* + * 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.delta + +import org.apache.gluten.sql.shims.SparkShimLoader + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.GlutenDeltaParquetFileFormat +import org.apache.spark.sql.delta.actions.DeletionVectorDescriptor +import org.apache.spark.sql.delta.deletionvectors.{RoaringBitmapArrayFormat, StoredBitmap} +import org.apache.spark.sql.delta.storage.dv.HadoopFileSystemDVStore +import org.apache.spark.sql.execution.datasources.PartitionedFile + +import org.apache.hadoop.fs.Path + +import java.util.{ArrayList => JArrayList} + +import scala.collection.JavaConverters._ +import scala.util.Try +import scala.util.control.NonFatal + +object DeltaDeletionVectorScanInfo { + object RowIndexFilterType extends Enumeration { + type RowIndexFilterType = Value + val KEEP_ALL, IF_CONTAINED, IF_NOT_CONTAINED = Value + } + + import RowIndexFilterType._ + + final case class DeletionVectorInfo( + rowIndexFilterType: RowIndexFilterType, + descriptor: Option[DeletionVectorDescriptor], + serializedDeletionVector: Array[Byte]) { + def hasDeletionVector: Boolean = descriptor.isDefined + def cardinality: Long = descriptor.map(_.cardinality).getOrElse(0L) + } + + final case class PartitionFileScanInfo( + normalizedOtherMetadataColumns: Map[String, Object], + deletionVectorInfo: DeletionVectorInfo) + + private val RowIndexFilterIdEncoded = + GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_ID_ENCODED + private val RowIndexFilterTypeKey = + GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_TYPE + + def extract( + spark: SparkSession, + partitionColumnCount: Int, + file: PartitionedFile): PartitionFileScanInfo = { + val metadata = otherMetadataColumns(file) + val normalizedMetadata = metadata -- Seq(RowIndexFilterIdEncoded, RowIndexFilterTypeKey) + val dvInfo = extractDeletionVectorInfo(spark, partitionColumnCount, file, metadata) + PartitionFileScanInfo(normalizedMetadata, dvInfo) + } + + def extractAll( + spark: SparkSession, + partitionColumnCount: Int, + files: Seq[PartitionedFile]): Seq[PartitionFileScanInfo] = { + files.map(extract(spark, partitionColumnCount, _)) + } + + def extractAllFromJava( + spark: SparkSession, + partitionColumnCount: Int, + files: java.util.List[PartitionedFile]): java.util.List[PartitionFileScanInfo] = { + new JArrayList(extractAll(spark, partitionColumnCount, files.asScala.toSeq).asJava) + } + + private def extractDeletionVectorInfo( + spark: SparkSession, + partitionColumnCount: Int, + file: PartitionedFile, + metadata: Map[String, Object]): DeletionVectorInfo = { + val descriptorValue = metadata.get(RowIndexFilterIdEncoded) + val filterTypeValue = metadata.get(RowIndexFilterTypeKey) + + (descriptorValue, filterTypeValue) match { + case (None, None) => + DeletionVectorInfo(KEEP_ALL, None, Array.emptyByteArray) + case (Some(encodedDescriptor), Some(filterType)) => + val descriptor = parseDescriptor(encodedDescriptor.toString) + val serializedPayload = serializePayload(spark, partitionColumnCount, file, descriptor) + DeletionVectorInfo( + parseRowIndexFilterType(filterType.toString), + Some(descriptor), + serializedPayload) + case _ => + throw new IllegalStateException( + s"Both $RowIndexFilterIdEncoded and $RowIndexFilterTypeKey must either be present or absent") + } + } + + private def otherMetadataColumns(file: PartitionedFile): Map[String, Object] = { + val otherMetadata = + SparkShimLoader.getSparkShims.getOtherConstantMetadataColumnValues(file) + if (otherMetadata == null) { + Map.empty + } else { + otherMetadata.asScala.toMap + } + } + + private def parseDescriptor(encodedDescriptor: String): DeletionVectorDescriptor = { + val methods = Seq("deserializeFromBase64", "fromJson") + methods.iterator + .map { + methodName => + Try { + val method = DeletionVectorDescriptor.getClass.getMethod(methodName, classOf[String]) + method + .invoke(DeletionVectorDescriptor, encodedDescriptor) + .asInstanceOf[DeletionVectorDescriptor] + }.toOption + } + .collectFirst { case Some(descriptor) => descriptor } + .getOrElse { + throw new IllegalArgumentException("Unable to parse Delta deletion vector descriptor") + } + } + + private def parseRowIndexFilterType(filterType: String): RowIndexFilterType = { + filterType match { + case "IF_CONTAINED" => IF_CONTAINED + case "IF_NOT_CONTAINED" => IF_NOT_CONTAINED + case "KEEP_ALL" => KEEP_ALL + case unexpected => + throw new IllegalStateException(s"Unexpected row index filter type: $unexpected") + } + } + + private def serializePayload( + spark: SparkSession, + partitionColumnCount: Int, + file: PartitionedFile, + descriptor: DeletionVectorDescriptor): Array[Byte] = { + val tablePath = resolveTablePath(spark, partitionColumnCount, file) + if (tablePath == null) { + throw new IllegalStateException( + "Unable to resolve Delta table path while materializing deletion vector payload") + } + val dvStore = new HadoopFileSystemDVStore(spark.sessionState.newHadoopConf()) + StoredBitmap + .create(descriptor, tablePath) + .load(dvStore) + .serializeAsByteArray(RoaringBitmapArrayFormat.Portable) + } + + private def resolveTablePath( + spark: SparkSession, + partitionColumnCount: Int, + file: PartitionedFile): Path = { + val fileParent = new Path(unescapePathName(file.filePath.toString)).getParent + var tablePath = fileParent + for (_ <- 0 until partitionColumnCount) { + tablePath = tablePath.getParent + } + if (tablePath != null && isDeltaTablePath(spark, tablePath)) { + return tablePath + } + + var candidate = fileParent + while (candidate != null && !isDeltaTablePath(spark, candidate)) { + candidate = candidate.getParent + } + if (candidate != null) candidate else tablePath + } + + private def isDeltaTablePath(spark: SparkSession, tablePath: Path): Boolean = { + val deltaLogPath = new Path(tablePath, "_delta_log") + try { + deltaLogPath.getFileSystem(spark.sessionState.newHadoopConf()).exists(deltaLogPath) + } catch { + case NonFatal(_) => false + } + } + + private def unescapePathName(path: String): String = { + if (path == null || path.indexOf('%') < 0) { + path + } else { + val builder = new StringBuilder(path.length) + var index = 0 + while (index < path.length) { + if (path.charAt(index) == '%' && index + 2 < path.length) { + val high = Character.digit(path.charAt(index + 1), 16) + val low = Character.digit(path.charAt(index + 2), 16) + if (high >= 0 && low >= 0) { + builder.append(((high << 4) | low).toChar) + index += 3 + } else { + builder.append(path.charAt(index)) + index += 1 + } + } else { + builder.append(path.charAt(index)) + index += 1 + } + } + builder.toString() + } + } +} diff --git a/backends-velox/src-delta40/test/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfoSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfoSuite.scala new file mode 100644 index 00000000000..cbb70817ba4 --- /dev/null +++ b/backends-velox/src-delta40/test/scala/org/apache/gluten/delta/DeltaDeletionVectorScanInfoSuite.scala @@ -0,0 +1,154 @@ +/* + * 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.delta + +import org.apache.gluten.delta.DeltaDeletionVectorScanInfo.RowIndexFilterType + +import org.apache.spark.SparkConf +import org.apache.spark.paths.SparkPath +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.delta.{DeltaLog, GlutenDeltaParquetFileFormat} +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import org.apache.spark.sql.delta.test.DeltaSQLTestUtils +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.tags.ExtendedSQLTest + +import io.delta.sql.DeltaSparkSessionExtension +import org.apache.hadoop.fs.Path + +@ExtendedSQLTest +class DeltaDeletionVectorScanInfoSuite + extends QueryTest + with SharedSparkSession + with DeltaSQLTestUtils { + + import testImplicits._ + + override protected def sparkConf: SparkConf = { + super.sparkConf + .set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, classOf[DeltaSparkSessionExtension].getName) + .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) + .set("spark.databricks.delta.snapshotPartitions", "2") + } + + test("extracts essential Delta DV scan info from split metadata") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)") + + val dataFile = DeltaLog + .forTable(spark, new Path(path)) + .update() + .allFiles + .collect() + .find(_.deletionVector != null) + .get + val partitionedFile = partitionedFileWithMetadata( + path, + dataFile.path, + dataFile.size, + Map( + GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_ID_ENCODED -> + dataFile.deletionVector.serializeToBase64(), + GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_TYPE -> "IF_CONTAINED", + "kept_key" -> "kept_value" + ) + ) + + val scanInfo = DeltaDeletionVectorScanInfo.extract(spark, 0, partitionedFile) + val dvInfo = scanInfo.deletionVectorInfo + + assert(dvInfo.hasDeletionVector) + assert(dvInfo.rowIndexFilterType == RowIndexFilterType.IF_CONTAINED) + assert(dvInfo.cardinality == dataFile.deletionVector.cardinality) + assert(dvInfo.serializedDeletionVector.nonEmpty) + assert(scanInfo.normalizedOtherMetadataColumns == Map("kept_key" -> "kept_value")) + } + } + + test("returns keep-all scan info when Delta DV metadata is absent") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a")).toDF("id", "value").coalesce(1).write.format("delta").save(path) + + val dataFile = DeltaLog.forTable(spark, new Path(path)).update().allFiles.collect().head + val partitionedFile = partitionedFileWithMetadata( + path, + dataFile.path, + dataFile.size, + Map("kept_key" -> "kept_value")) + + val scanInfo = DeltaDeletionVectorScanInfo.extract(spark, 0, partitionedFile) + val dvInfo = scanInfo.deletionVectorInfo + + assert(!dvInfo.hasDeletionVector) + assert(dvInfo.rowIndexFilterType == RowIndexFilterType.KEEP_ALL) + assert(dvInfo.cardinality == 0L) + assert(dvInfo.serializedDeletionVector.isEmpty) + assert(scanInfo.normalizedOtherMetadataColumns == Map("kept_key" -> "kept_value")) + } + } + + test("rejects partial Delta DV split metadata") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a")).toDF("id", "value").coalesce(1).write.format("delta").save(path) + + val dataFile = DeltaLog.forTable(spark, new Path(path)).update().allFiles.collect().head + val partitionedFile = partitionedFileWithMetadata( + path, + dataFile.path, + dataFile.size, + Map(GlutenDeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_TYPE -> "IF_CONTAINED")) + + val error = intercept[IllegalStateException] { + DeltaDeletionVectorScanInfo.extract(spark, 0, partitionedFile) + } + assert(error.getMessage.contains("must either be present or absent")) + } + } + + private def partitionedFileWithMetadata( + tablePath: String, + relativeFilePath: String, + fileSize: Long, + metadata: Map[String, Object]): PartitionedFile = { + PartitionedFile( + partitionValues = InternalRow.empty, + filePath = SparkPath.fromPath(new Path(tablePath, relativeFilePath)), + start = 0L, + length = fileSize, + fileSize = fileSize, + otherConstantMetadataColumnValues = metadata + ) + } +} From c1fe399b045f809462266eff9dab58c943c8877a Mon Sep 17 00:00:00 2001 From: Mohammad Linjawi Date: Sun, 31 May 2026 12:22:09 +0300 Subject: [PATCH 02/16] [VL][Delta] Add JVM DV scan handoff --- .../component/VeloxDeltaComponent.scala | 25 ++ ...eprocessDeltaScanWithDeletionVectors.scala | 36 +++ .../sql/delta/PreprocessTableWithDVs.scala | 228 +++++++++++++++ .../DeltaDeletionVectorHandoffSuite.scala | 68 +++++ .../apache/spark/sql/delta/DeltaSuite.scala | 28 ++ .../DeltaDeletionVectorHandoffSuite.scala | 98 +++++++ .../backendsapi/velox/VeloxIteratorApi.scala | 132 ++++++++- cpp/velox/compute/VeloxPlanConverter.cc | 87 ++++++ cpp/velox/compute/WholeStageResultIterator.cc | 85 +++++- cpp/velox/compute/delta/DeltaSplitInfo.h | 46 +++ cpp/velox/substrait/SubstraitToVeloxPlan.cc | 27 +- .../execution/DeltaScanTransformer.scala | 13 - .../extension/DeltaPostTransformRules.scala | 274 ++++++++++++------ .../gluten/extension/OffloadDeltaScan.scala | 79 ++++- .../apache/gluten/execution/DeltaSuite.scala | 22 +- .../substrait/rel/DeltaLocalFilesBuilder.java | 56 ++++ .../substrait/rel/DeltaLocalFilesNode.java | 132 +++++++++ .../substrait/proto/substrait/algebra.proto | 13 + 18 files changed, 1325 insertions(+), 124 deletions(-) create mode 100644 backends-velox/src-delta33/main/scala/org/apache/gluten/extension/PreprocessDeltaScanWithDeletionVectors.scala create mode 100644 backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/PreprocessTableWithDVs.scala create mode 100644 backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala create mode 100644 backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala create mode 100644 cpp/velox/compute/delta/DeltaSplitInfo.h create mode 100644 gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/DeltaLocalFilesBuilder.java create mode 100644 gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/DeltaLocalFilesNode.java diff --git a/backends-velox/src-delta/main/scala/org/apache/gluten/component/VeloxDeltaComponent.scala b/backends-velox/src-delta/main/scala/org/apache/gluten/component/VeloxDeltaComponent.scala index 0587e8b07f7..4c6254d273d 100644 --- a/backends-velox/src-delta/main/scala/org/apache/gluten/component/VeloxDeltaComponent.scala +++ b/backends-velox/src-delta/main/scala/org/apache/gluten/component/VeloxDeltaComponent.scala @@ -23,9 +23,15 @@ import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.extension.columnar.validator.Validators import org.apache.gluten.extension.injector.Injector +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.SparkReflectionUtil class VeloxDeltaComponent extends Component { + private val deltaDvPreprocessRuleClassName = + "org.apache.gluten.extension.PreprocessDeltaScanWithDeletionVectors" + override def name(): String = "velox-delta" override def dependencies(): Seq[Class[_ <: Component]] = classOf[VeloxBackend] :: Nil @@ -36,6 +42,7 @@ class VeloxDeltaComponent extends Component { override def injectRules(injector: Injector): Unit = { val legacy = injector.gluten.legacy + injector.spark.injectOptimizerRule(deltaDvPreprocessRule) legacy.injectTransform { c => val offload = Seq(OffloadDeltaScan(), OffloadDeltaProject(), OffloadDeltaFilter()) @@ -46,4 +53,22 @@ class VeloxDeltaComponent extends Component { } DeltaPostTransformRules.rules.foreach(r => legacy.injectPostTransform(_ => r)) } + + private def deltaDvPreprocessRule(spark: SparkSession): Rule[LogicalPlan] = { + if (!SparkReflectionUtil.isClassPresent(deltaDvPreprocessRuleClassName)) { + return VeloxDeltaComponent.IdentityRule + } + + Class + .forName(deltaDvPreprocessRuleClassName) + .getConstructor(classOf[SparkSession]) + .newInstance(spark) + .asInstanceOf[Rule[LogicalPlan]] + } +} + +object VeloxDeltaComponent { + private object IdentityRule extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan + } } diff --git a/backends-velox/src-delta33/main/scala/org/apache/gluten/extension/PreprocessDeltaScanWithDeletionVectors.scala b/backends-velox/src-delta33/main/scala/org/apache/gluten/extension/PreprocessDeltaScanWithDeletionVectors.scala new file mode 100644 index 00000000000..26a98865627 --- /dev/null +++ b/backends-velox/src-delta33/main/scala/org/apache/gluten/extension/PreprocessDeltaScanWithDeletionVectors.scala @@ -0,0 +1,36 @@ +/* + * 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.extension + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.delta.PreprocessTableWithDVs + +/** + * Delta 3.3 compatibility rule for DV scan metadata. + * + * Delta's own PrepareDeltaScan still runs normally. This Gluten-scoped rule only adds the + * backend-visible DV metadata columns after Delta has prepared the scan, so the physical Delta scan + * handoff can materialize the per-file DV payload for Velox without replacing Delta classes. + */ +class PreprocessDeltaScanWithDeletionVectors(protected val spark: SparkSession) + extends Rule[LogicalPlan] + with PreprocessTableWithDVs { + + override def apply(plan: LogicalPlan): LogicalPlan = preprocessTablesWithDVs(plan) +} diff --git a/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/PreprocessTableWithDVs.scala b/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/PreprocessTableWithDVs.scala new file mode 100644 index 00000000000..56bae355005 --- /dev/null +++ b/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/PreprocessTableWithDVs.scala @@ -0,0 +1,228 @@ +/* + * 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 + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.delta.DeltaParquetFileFormat._ +import org.apache.spark.sql.delta.commands.DeletionVectorUtils.deletionVectorsReadable +import org.apache.spark.sql.delta.files.{TahoeFileIndex, TahoeLogFileIndex} +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.execution.datasources.FileFormat.METADATA_NAME +import org.apache.spark.sql.execution.datasources.HadoopFsRelation +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.types.StructType + +/** + * Rewrites Delta scans over DV-enabled tables to request the backend-specific skip-row metadata + * column only when the snapshot actually contains DVs. + */ +trait PreprocessTableWithDVs extends SubqueryTransformerHelper { + def preprocessTablesWithDVs(plan: LogicalPlan): LogicalPlan = { + plan.transformDown { case ScanWithDeletionVectors(dvScan) => dvScan } + } +} + +object ScanWithDeletionVectors { + def unapply(a: LogicalRelation): Option[LogicalPlan] = a match { + case scan @ LogicalRelation( + relation @ HadoopFsRelation( + index: TahoeFileIndex, + _, + _, + _, + format: DeltaParquetFileFormat, + _), + _, + _, + _) => + dvEnabledScanFor(scan, relation, format, index) + case scan @ LogicalRelation( + relation @ HadoopFsRelation( + index: TahoeFileIndex, + _, + _, + _, + format: GlutenDeltaParquetFileFormat, + _), + _, + _, + _) => + dvEnabledScanFor(scan, relation, format, index) + case _ => None + } + + def dvEnabledScanFor( + scan: LogicalRelation, + hadoopRelation: HadoopFsRelation, + fileFormat: DeltaParquetFileFormat, + index: TahoeFileIndex): Option[LogicalPlan] = { + if (!deletionVectorsReadable(index.protocol, index.metadata)) { + return None + } + + if (index.isInstanceOf[TahoeLogFileIndex]) { + return None + } + + if (fileFormat.hasTablePath) { + return None + } + + val filesWithDVs = index + .matchingFiles(partitionFilters = Seq(TrueLiteral), dataFilters = Seq(TrueLiteral)) + .filter(_.deletionVector != null) + if (filesWithDVs.isEmpty) { + return None + } + + val planOutput = scan.output + val spark = SparkSession.getActiveSession.get + val newScan = createScanWithSkipRowColumn(spark, scan, fileFormat, index, hadoopRelation) + val rowIndexFilter = createRowIndexFilterNode(newScan) + Some(Project(planOutput, rowIndexFilter)) + } + + def dvEnabledScanFor( + scan: LogicalRelation, + hadoopRelation: HadoopFsRelation, + fileFormat: GlutenDeltaParquetFileFormat, + index: TahoeFileIndex): Option[LogicalPlan] = { + if (!deletionVectorsReadable(index.protocol, index.metadata)) { + return None + } + + if (index.isInstanceOf[TahoeLogFileIndex]) { + return None + } + + if (fileFormat.hasTablePath) { + return None + } + + val filesWithDVs = index + .matchingFiles(partitionFilters = Seq(TrueLiteral), dataFilters = Seq(TrueLiteral)) + .filter(_.deletionVector != null) + if (filesWithDVs.isEmpty) { + return None + } + + val planOutput = scan.output + val spark = SparkSession.getActiveSession.get + val newScan = createScanWithSkipRowColumn(spark, scan, fileFormat, index, hadoopRelation) + val rowIndexFilter = createRowIndexFilterNode(newScan) + Some(Project(planOutput, rowIndexFilter)) + } + + private def addRowIndexIfMissing(attribute: AttributeReference): AttributeReference = { + require(attribute.name == METADATA_NAME) + + val dataType = attribute.dataType.asInstanceOf[StructType] + if (dataType.fieldNames.contains(ParquetFileFormat.ROW_INDEX)) { + return attribute + } + + val newDatatype = dataType.add(ParquetFileFormat.ROW_INDEX_FIELD) + attribute.copy(dataType = newDatatype)( + exprId = attribute.exprId, + qualifier = attribute.qualifier) + } + + private def createScanWithSkipRowColumn( + spark: SparkSession, + inputScan: LogicalRelation, + fileFormat: DeltaParquetFileFormat, + tahoeFileIndex: TahoeFileIndex, + hadoopFsRelation: HadoopFsRelation): LogicalRelation = { + val useMetadataRowIndex = + spark.sessionState.conf.getConf(DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX) + + val skipRowField = IS_ROW_DELETED_STRUCT_FIELD + val scanOutputWithMetadata = if (useMetadataRowIndex) { + if (inputScan.output.map(_.name).contains(METADATA_NAME)) { + inputScan.output.collect { + case a: AttributeReference if a.name == METADATA_NAME => addRowIndexIfMissing(a) + case o => o + } + } else { + inputScan.output :+ fileFormat.createFileMetadataCol() + } + } else { + inputScan.output + } + + val newScanOutput = + scanOutputWithMetadata :+ AttributeReference(skipRowField.name, skipRowField.dataType)() + val newDataSchema = hadoopFsRelation.dataSchema.add(skipRowField) + val newFileFormat = fileFormat.copyWithDVInfo( + tablePath = tahoeFileIndex.path.toString, + optimizationsEnabled = useMetadataRowIndex) + + val newRelation = hadoopFsRelation.copy(fileFormat = newFileFormat, dataSchema = newDataSchema)( + hadoopFsRelation.sparkSession) + + inputScan.copy(relation = newRelation, output = newScanOutput) + } + + private def createScanWithSkipRowColumn( + spark: SparkSession, + inputScan: LogicalRelation, + fileFormat: GlutenDeltaParquetFileFormat, + tahoeFileIndex: TahoeFileIndex, + hadoopFsRelation: HadoopFsRelation): LogicalRelation = { + val useMetadataRowIndex = + spark.sessionState.conf.getConf(DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX) + + val skipRowField = GlutenDeltaParquetFileFormat.IS_ROW_DELETED_STRUCT_FIELD + val scanOutputWithMetadata = if (useMetadataRowIndex) { + if (inputScan.output.map(_.name).contains(METADATA_NAME)) { + inputScan.output.collect { + case a: AttributeReference if a.name == METADATA_NAME => addRowIndexIfMissing(a) + case o => o + } + } else { + inputScan.output :+ fileFormat.createFileMetadataCol() + } + } else { + inputScan.output + } + + val newScanOutput = + scanOutputWithMetadata :+ AttributeReference(skipRowField.name, skipRowField.dataType)() + val newDataSchema = hadoopFsRelation.dataSchema.add(skipRowField) + val newFileFormat = fileFormat.copyWithDVInfo( + tablePath = tahoeFileIndex.path.toString, + optimizationsEnabled = useMetadataRowIndex) + + val newRelation = hadoopFsRelation.copy(fileFormat = newFileFormat, dataSchema = newDataSchema)( + hadoopFsRelation.sparkSession) + + inputScan.copy(relation = newRelation, output = newScanOutput) + } + + private def createRowIndexFilterNode(newScan: LogicalRelation): Filter = { + val skipRowColumnRefs = newScan.output.filter(_.name == IS_ROW_DELETED_COLUMN_NAME) + require( + skipRowColumnRefs.size == 1, + s"Expected only one column with name=$IS_ROW_DELETED_COLUMN_NAME") + val skipRowColumnRef = skipRowColumnRefs.head + Filter(EqualTo(skipRowColumnRef, Literal(RowIndexFilter.KEEP_ROW_VALUE)), newScan) + } +} diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala new file mode 100644 index 00000000000..f5510a95255 --- /dev/null +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -0,0 +1,68 @@ +/* + * 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 + +import org.apache.gluten.execution.DeltaScanTransformer + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.delta.test.{DeltaSQLCommandTest, DeltaSQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.tags.ExtendedSQLTest + +import org.apache.hadoop.fs.Path + +@ExtendedSQLTest +class DeltaDeletionVectorHandoffSuite + extends QueryTest + with SharedSparkSession + with DeltaSQLTestUtils + with DeltaSQLCommandTest { + + import testImplicits._ + + test("Spark 3.5 Delta DV scan handoff should filter deleted rows") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)") + + val log = DeltaLog.forTable(spark, new Path(path)) + val addFileWithDv = log.update().allFiles.collect().find(_.deletionVector != null) + assert(addFileWithDv.nonEmpty) + + val dataFile = addFileWithDv.get + assert(dataFile.deletionVector.cardinality == 2L) + + val df = spark.read.format("delta").load(path) + val executedPlan = df.queryExecution.executedPlan + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.nonEmpty) + val planText = executedPlan.toString() + assert(!planText.contains("__delta_internal_is_row_deleted")) + assert(!planText.contains("__delta_internal_row_index")) + checkAnswer(df, Seq((1, "a"), (2, "b")).toDF()) + } + } +} diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala index f265168ddbd..a60054031c6 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaSuite.scala @@ -197,6 +197,34 @@ class DeltaSuite checkAnswer(data.toDF(), Row(1) :: Row(2) :: Row(3) :: Row(4) :: Row(5) :: Row(6) :: Nil) } + test("DV scan without metadata row index falls back and stays correct") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + + withSQLConf(DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> "false") { + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)") + + val log = DeltaLog.forTable(spark, new Path(path)) + assert(log.update().allFiles.collect().exists(_.deletionVector != null)) + + val df = spark.read.format("delta").load(path) + val executedPlan = df.queryExecution.executedPlan + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.isEmpty) + checkAnswer(df, Seq(Row(1, "a"), Row(2, "b"))) + } + } + } + test("partitioned append - nulls") { val tempDir = Utils.createTempDir() Seq(Some(1), None) diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala new file mode 100644 index 00000000000..dda547b015f --- /dev/null +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -0,0 +1,98 @@ +/* + * 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 + +import org.apache.gluten.execution.DeltaScanTransformer + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.delta.test.{DeltaSQLCommandTest, DeltaSQLTestUtils} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.tags.ExtendedSQLTest + +import org.apache.hadoop.fs.Path + +@ExtendedSQLTest +class DeltaDeletionVectorHandoffSuite + extends QueryTest + with SharedSparkSession + with DeltaSQLTestUtils + with DeltaSQLCommandTest { + + import testImplicits._ + + test("Spark 4 Delta DV scan should fall back when metadata row index is disabled") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)") + + val log = DeltaLog.forTable(spark, new Path(path)) + assert(log.update().allFiles.collect().exists(_.deletionVector != null)) + + // This covers scan behavior over an existing DV. Keep the no-metadata-row-index + // path on Spark until the native path can prove the same contract for DML DVs. + withSQLConf(DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> "false") { + val df = spark.read.format("delta").load(path) + val executedPlan = df.queryExecution.executedPlan + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.isEmpty) + checkAnswer(df, Seq((1, "a"), (2, "b")).toDF()) + } + } + } + + test("Spark 4 Delta DV scan handoff should filter deleted rows") { + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)") + + val log = DeltaLog.forTable(spark, new Path(path)) + val addFileWithDv = log.update().allFiles.collect().find(_.deletionVector != null) + assert(addFileWithDv.nonEmpty) + + val dataFile = addFileWithDv.get + assert(dataFile.deletionVector.cardinality == 2L) + + val df = spark.read.format("delta").load(path) + val executedPlan = df.queryExecution.executedPlan + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.nonEmpty) + val planText = executedPlan.toString() + assert(!planText.contains("__delta_internal_is_row_deleted")) + assert(!planText.contains("__delta_internal_row_index")) + checkAnswer(df, Seq((1, "a"), (2, "b")).toDF()) + } + } +} diff --git a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala index d8b23b358fa..1cf04ba5423 100644 --- a/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala +++ b/backends-velox/src/main/scala/org/apache/gluten/backendsapi/velox/VeloxIteratorApi.scala @@ -24,13 +24,15 @@ import org.apache.gluten.iterator.Iterators import org.apache.gluten.metrics.{IMetrics, IteratorMetricsJniWrapper} import org.apache.gluten.sql.shims.SparkShimLoader import org.apache.gluten.substrait.plan.PlanNode -import org.apache.gluten.substrait.rel.{LocalFilesBuilder, LocalFilesNode, SplitInfo} +import org.apache.gluten.substrait.rel.{DeltaLocalFilesBuilder, LocalFilesBuilder, LocalFilesNode, SplitInfo} +import org.apache.gluten.substrait.rel.DeltaLocalFilesNode.{DeltaFileReadOptions, RowIndexFilterType} import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat import org.apache.gluten.vectorized._ import org.apache.spark.{Partition, SparkConf, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.softaffinity.SoftAffinity +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile} @@ -49,6 +51,10 @@ import scala.collection.JavaConverters._ import scala.collection.mutable class VeloxIteratorApi extends IteratorApi with Logging { + private type NormalizedDeltaSplitMetadata = + (Seq[java.util.Map[String, Object]], Seq[DeltaFileReadOptions]) + + private val deltaScanInfoClassName = "org.apache.gluten.delta.DeltaDeletionVectorScanInfo$" private def setFileSchemaForLocalFiles( localFilesNode: LocalFilesNode, @@ -94,10 +100,33 @@ class VeloxIteratorApi extends IteratorApi with Logging { val metadataColumns = partitionFiles .map( f => SparkShimLoader.getSparkShims.generateMetadataColumns(f, metadataColumnNames).asJava) - val otherMetadataColumns = partitionFiles - .map(f => SparkShimLoader.getSparkShims.getOtherConstantMetadataColumnValues(f)) + val (otherMetadataColumns, deltaReadOptions) = + normalizeDeltaSplitMetadata(partitionSchema.fields.length, partitionFiles) + .getOrElse { + ( + partitionFiles.map { + f => SparkShimLoader.getSparkShims.getOtherConstantMetadataColumnValues(f) + }, + Seq.empty[DeltaFileReadOptions]) + } - setFileSchemaForLocalFiles( + val localFilesNode = if (deltaReadOptions.nonEmpty) { + DeltaLocalFilesBuilder.makeDeltaLocalFiles( + partitionIndex, + paths.asJava, + starts.asJava, + lengths.asJava, + fileSizes.asJava, + modificationTimes.asJava, + partitionColumns.map(_.asJava).asJava, + metadataColumns.asJava, + fileFormat, + locations.toList.asJava, + mapAsJavaMap(properties), + otherMetadataColumns.asJava, + deltaReadOptions.asJava + ) + } else { LocalFilesBuilder.makeLocalFiles( partitionIndex, paths.asJava, @@ -111,10 +140,16 @@ class VeloxIteratorApi extends IteratorApi with Logging { locations.toList.asJava, mapAsJavaMap(properties), otherMetadataColumns.asJava - ), + ) + } + + val localFiles = setFileSchemaForLocalFiles( + localFilesNode, dataSchema, fileFormat ) + + localFiles } /** Generate native row partition. */ @@ -179,6 +214,93 @@ class VeloxIteratorApi extends IteratorApi with Logging { NativePlanEvaluator.injectWriteFilesTempPath(path, fileName) } + private def normalizeDeltaSplitMetadata( + partitionColumnCount: Int, + partitionFiles: Seq[PartitionedFile]): Option[NormalizedDeltaSplitMetadata] = { + try { + // scalastyle:off classforname + val moduleClass = Class.forName(deltaScanInfoClassName) + // scalastyle:on classforname + val module = moduleClass.getField("MODULE$").get(null) + val extractAllMethod = moduleClass.getMethod( + "extractAllFromJava", + classOf[SparkSession], + classOf[Int], + classOf[java.util.List[_]]) + val scanInfos = extractAllMethod + .invoke(module, activeSparkSession, Int.box(partitionColumnCount), partitionFiles.asJava) + .asInstanceOf[java.util.List[_]] + .asScala + .toSeq + val splitMetadata = scanInfos.map(toDeltaSplitMetadata) + if (splitMetadata.exists(_._2.hasDeletionVector())) { + Some((splitMetadata.map(_._1), splitMetadata.map(_._2))) + } else { + None + } + } catch { + case _: ClassNotFoundException | _: NoSuchMethodException => + None + } + } + + private def toDeltaSplitMetadata( + scanInfo: Any): (java.util.Map[String, Object], DeltaFileReadOptions) = { + val metadata = scanInfo + .getClass + .getMethod("normalizedOtherMetadataColumns") + .invoke(scanInfo) + .asInstanceOf[scala.collection.Map[String, Object]] + .asJava + val deletionVectorInfo = scanInfo.getClass.getMethod("deletionVectorInfo").invoke(scanInfo) + val rowIndexFilterType = deletionVectorInfo + .getClass + .getMethod("rowIndexFilterType") + .invoke(deletionVectorInfo) + .toString + val hasDeletionVector = deletionVectorInfo + .getClass + .getMethod("hasDeletionVector") + .invoke(deletionVectorInfo) + .asInstanceOf[Boolean] + val cardinality = deletionVectorInfo + .getClass + .getMethod("cardinality") + .invoke(deletionVectorInfo) + .asInstanceOf[JLong] + .longValue() + val serializedDeletionVector = deletionVectorInfo + .getClass + .getMethod("serializedDeletionVector") + .invoke(deletionVectorInfo) + .asInstanceOf[Array[Byte]] + + ( + metadata, + new DeltaFileReadOptions( + toDeltaRowIndexFilterType(rowIndexFilterType), + hasDeletionVector, + cardinality, + serializedDeletionVector)) + } + + private def toDeltaRowIndexFilterType(rowIndexFilterType: String): RowIndexFilterType = { + rowIndexFilterType match { + case "IF_CONTAINED" => RowIndexFilterType.IF_CONTAINED + case "IF_NOT_CONTAINED" => RowIndexFilterType.IF_NOT_CONTAINED + case _ => RowIndexFilterType.KEEP_ALL + } + } + + private def activeSparkSession: SparkSession = { + SparkSession.getActiveSession + .orElse(SparkSession.getDefaultSession) + .getOrElse { + throw new IllegalStateException( + "Active SparkSession is required to materialize Delta deletion vectors") + } + } + /** Generate Iterator[ColumnarBatch] for first stage. */ override def genFirstStageIterator( inputPartition: BaseGlutenPartition, diff --git a/cpp/velox/compute/VeloxPlanConverter.cc b/cpp/velox/compute/VeloxPlanConverter.cc index f3ffab59a6a..befa4e9dbd1 100644 --- a/cpp/velox/compute/VeloxPlanConverter.cc +++ b/cpp/velox/compute/VeloxPlanConverter.cc @@ -17,8 +17,13 @@ #include "VeloxPlanConverter.h" #include +#include +#include +#include +#include #include "config/GlutenConfig.h" +#include "delta/DeltaSplitInfo.h" #include "iceberg/IcebergPlanConverter.h" #include "operators/plannodes/IteratorSplit.h" @@ -48,6 +53,80 @@ VeloxPlanConverter::VeloxPlanConverter( } namespace { +std::optional unpackMetadataValue(const google::protobuf::Any& value) { + google::protobuf::BytesValue bytesValue; + if (value.UnpackTo(&bytesValue)) { + return bytesValue.value(); + } + + google::protobuf::StringValue stringValue; + if (value.UnpackTo(&stringValue)) { + return stringValue.value(); + } + + google::protobuf::Int32Value int32Value; + if (value.UnpackTo(&int32Value)) { + return std::to_string(int32Value.value()); + } + + google::protobuf::Int64Value int64Value; + if (value.UnpackTo(&int64Value)) { + return std::to_string(int64Value.value()); + } + + google::protobuf::DoubleValue doubleValue; + if (value.UnpackTo(&doubleValue)) { + return std::to_string(doubleValue.value()); + } + + return std::nullopt; +} + +delta::DeltaRowIndexFilterType parseDeltaRowIndexFilterType(int filterType) { + switch (filterType) { + case 1: + return delta::DeltaRowIndexFilterType::kIfContained; + case 2: + return delta::DeltaRowIndexFilterType::kIfNotContained; + case 0: + default: + return delta::DeltaRowIndexFilterType::kKeepAll; + } +} + +std::shared_ptr parseDeltaSplitInfo( + const substrait::ReadRel_LocalFiles_FileOrFiles& file, + std::shared_ptr splitInfo) { + auto deltaSplitInfo = std::dynamic_pointer_cast(splitInfo) + ? std::dynamic_pointer_cast(splitInfo) + : std::make_shared(*splitInfo); + + deltaSplitInfo->format = dwio::common::FileFormat::PARQUET; + const auto& deltaReadOptions = file.delta(); + deltaSplitInfo->rowIndexFilterTypes.emplace_back( + parseDeltaRowIndexFilterType(deltaReadOptions.row_index_filter_type())); + + if (!deltaReadOptions.has_deletion_vector()) { + deltaSplitInfo->deletionVectors.emplace_back(std::nullopt); + return deltaSplitInfo; + } + + auto serializedPayload = deltaReadOptions.serialized_deletion_vector(); + VELOX_USER_CHECK(!serializedPayload.empty(), "Delta split has a deletion vector without a serialized payload"); + VELOX_USER_CHECK_LE( + serializedPayload.size(), + static_cast(std::numeric_limits::max()), + "Delta deletion vector serialized payload is too large"); + const auto cardinality = static_cast(deltaReadOptions.deletion_vector_cardinality()); + auto payload = std::make_shared(std::move(serializedPayload)); + const SplitPayloadBufferView payloadView{ + reinterpret_cast(payload->data()), static_cast(payload->size())}; + deltaSplitInfo->deletionVectors.emplace_back( + delta::DeltaDeletionVectorDescriptor::serialized(cardinality, payloadView)); + deltaSplitInfo->deletionVectorPayloads.emplace_back(std::move(payload)); + return deltaSplitInfo; +} + std::shared_ptr parseScanSplitInfo( const facebook::velox::config::ConfigBase* veloxCfg, const google::protobuf::RepeatedPtrField& fileList) { @@ -75,6 +154,11 @@ std::shared_ptr parseScanSplitInfo( for (const auto& metadataColumn : file.metadata_columns()) { metadataColumnMap[metadataColumn.key()] = metadataColumn.value(); } + for (const auto& otherMetadataColumn : file.other_const_metadata_columns()) { + if (auto unpackedValue = unpackMetadataValue(otherMetadataColumn.value())) { + metadataColumnMap[otherMetadataColumn.key()] = std::move(*unpackedValue); + } + } splitInfo->metadataColumns.emplace_back(metadataColumnMap); splitInfo->paths.emplace_back(file.uri_file()); @@ -103,6 +187,9 @@ std::shared_ptr parseScanSplitInfo( case SubstraitFileFormatCase::kIceberg: splitInfo = IcebergPlanConverter::parseIcebergSplitInfo(file, std::move(splitInfo)); break; + case SubstraitFileFormatCase::kDelta: + splitInfo = parseDeltaSplitInfo(file, std::move(splitInfo)); + break; default: splitInfo->format = dwio::common::FileFormat::UNKNOWN; break; diff --git a/cpp/velox/compute/WholeStageResultIterator.cc b/cpp/velox/compute/WholeStageResultIterator.cc index c3ac095cdc7..8499d56b352 100644 --- a/cpp/velox/compute/WholeStageResultIterator.cc +++ b/cpp/velox/compute/WholeStageResultIterator.cc @@ -15,9 +15,13 @@ * limitations under the License. */ #include "WholeStageResultIterator.h" +#include #include "VeloxBackend.h" #include "VeloxPlanConverter.h" #include "VeloxRuntime.h" +#include "compute/delta/DeltaConnector.h" +#include "compute/delta/DeltaSplit.h" +#include "compute/delta/DeltaSplitInfo.h" #include "config/VeloxConfig.h" #include "utils/ConfigExtractor.h" #include "velox/connectors/hive/HiveConfig.h" @@ -69,6 +73,51 @@ const std::string kWriteIOTime = "writeIOWallNanos"; // others const std::string kHiveDefaultPartition = "__HIVE_DEFAULT_PARTITION__"; +const std::string kDeltaTableFormat = "delta"; +const std::string kTableFormatKey = "table_format"; + +bool isDeltaMetadata(const std::unordered_map& metadata) { + auto tableFormatIt = metadata.find(kTableFormatKey); + return tableFormatIt != metadata.end() && tableFormatIt->second == kDeltaTableFormat; +} + +bool isDeltaScanInfo(const std::shared_ptr& splitInfo) { + for (const auto& metadata : splitInfo->metadataColumns) { + if (isDeltaMetadata(metadata)) { + return true; + } + } + return false; +} + +const velox::core::TableScanNode* findTableScanNodeById( + const std::shared_ptr& planNode, + const velox::core::PlanNodeId& nodeId) { + if (planNode == nullptr) { + return nullptr; + } + + if (planNode->id() == nodeId) { + return dynamic_cast(planNode.get()); + } + + for (const auto& source : planNode->sources()) { + if (const auto* found = findTableScanNodeById(source, nodeId)) { + return found; + } + } + return nullptr; +} + +std::string connectorIdForScanNode( + const std::shared_ptr& planNode, + const velox::core::PlanNodeId& nodeId) { + const auto* tableScanNode = findTableScanNodeById(planNode, nodeId); + if (tableScanNode == nullptr) { + return ""; + } + return tableScanNode->tableHandle()->connectorId(); +} } // namespace @@ -134,7 +183,8 @@ WholeStageResultIterator::WholeStageResultIterator( throw std::runtime_error("Invalid scan information."); } - for (const auto& scanInfo : scanInfos) { + for (size_t scanInfoIdx = 0; scanInfoIdx < scanInfos.size(); ++scanInfoIdx) { + const auto& scanInfo = scanInfos[scanInfoIdx]; // Get the information for TableScan. // Partition index in scan info is not used. const auto& paths = scanInfo->paths; @@ -144,6 +194,10 @@ WholeStageResultIterator::WholeStageResultIterator( const auto& format = scanInfo->format; const auto& partitionColumns = scanInfo->partitionColumns; const auto& metadataColumns = scanInfo->metadataColumns; + const auto scanNodeConnectorId = connectorIdForScanNode(veloxPlan_, scanNodeIds_[scanInfoIdx]); + const auto deltaSplitInfo = std::dynamic_pointer_cast(scanInfo); + const bool isDeltaScan = + scanNodeConnectorId == connectorIds_.delta || deltaSplitInfo != nullptr || isDeltaScanInfo(scanInfo); #ifdef GLUTEN_ENABLE_GPU // Under the pre-condition that all the split infos has same partition column and format. const auto canUseCudfConnector = scanInfo->canUseCudfConnector(); @@ -177,10 +231,37 @@ WholeStageResultIterator::WholeStageResultIterator( deleteFiles, metadataColumn, properties[idx]); + } else if (isDeltaScan) { + std::unordered_map customSplitInfo{{"table_format", kDeltaTableFormat}}; + std::optional deletionVector = std::nullopt; + auto rowIndexFilterType = gluten::delta::DeltaRowIndexFilterType::kKeepAll; + if (deltaSplitInfo != nullptr) { + VELOX_USER_CHECK_LT(idx, deltaSplitInfo->deletionVectors.size()); + VELOX_USER_CHECK_LT(idx, deltaSplitInfo->rowIndexFilterTypes.size()); + deletionVector = deltaSplitInfo->deletionVectors[idx]; + rowIndexFilterType = deltaSplitInfo->rowIndexFilterTypes[idx]; + } + split = std::make_shared( + connectorIds_.delta, + paths[idx], + format, + starts[idx], + lengths[idx], + partitionKeys, + std::nullopt, + customSplitInfo, + nullptr, + std::unordered_map(), + true, + deletionVector, + std::nullopt, + rowIndexFilterType, + metadataColumn, + properties[idx]); } else { auto connectorId = connectorIds_.hive; #ifdef GLUTEN_ENABLE_GPU - if (canUseCudfConnector && enableCudf_ && + if (connectorId == connectorIds_.hive && canUseCudfConnector && enableCudf_ && veloxCfg_->get(kCudfEnableTableScan, kCudfEnableTableScanDefault)) { connectorId = connectorIds_.cudfHive; } diff --git a/cpp/velox/compute/delta/DeltaSplitInfo.h b/cpp/velox/compute/delta/DeltaSplitInfo.h new file mode 100644 index 00000000000..c02e52f6b88 --- /dev/null +++ b/cpp/velox/compute/delta/DeltaSplitInfo.h @@ -0,0 +1,46 @@ +/* + * 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. + */ + +#pragma once + +#include +#include +#include +#include + +#include "compute/delta/DeltaSplit.h" +#include "substrait/SubstraitToVeloxPlan.h" + +namespace gluten { + +struct DeltaSplitInfo : SplitInfo { + std::vector> deletionVectorPayloads; + std::vector> deletionVectors; + std::vector rowIndexFilterTypes; + + DeltaSplitInfo(const SplitInfo& splitInfo) : SplitInfo(splitInfo) { + deletionVectors.reserve(splitInfo.paths.capacity()); + deletionVectorPayloads.reserve(splitInfo.paths.capacity()); + rowIndexFilterTypes.reserve(splitInfo.paths.capacity()); + + const auto previousFileCount = splitInfo.paths.empty() ? 0 : splitInfo.paths.size() - 1; + deletionVectors.resize(previousFileCount, std::nullopt); + rowIndexFilterTypes.resize(previousFileCount, delta::DeltaRowIndexFilterType::kKeepAll); + } +}; + +} // namespace gluten diff --git a/cpp/velox/substrait/SubstraitToVeloxPlan.cc b/cpp/velox/substrait/SubstraitToVeloxPlan.cc index 5477176ce85..7d577c12a63 100644 --- a/cpp/velox/substrait/SubstraitToVeloxPlan.cc +++ b/cpp/velox/substrait/SubstraitToVeloxPlan.cc @@ -19,6 +19,8 @@ #include "TypeUtils.h" #include "VariantToVectorConverter.h" +#include "compute/delta/DeltaConnector.h" +#include "compute/delta/DeltaSplitInfo.h" #include "jni/JniHashTable.h" #include "operators/hashjoin/HashTableBuilder.h" #include "operators/plannodes/RowVectorStream.h" @@ -46,6 +48,9 @@ using namespace cudf_velox::connector::hive; namespace gluten { namespace { +const std::string kDeltaTableFormat = "delta"; +const std::string kTableFormatKey = "table_format"; + bool useCudfTableHandle(const std::vector>& splitInfos) { #ifdef GLUTEN_ENABLE_GPU if (splitInfos.empty()) { @@ -57,6 +62,23 @@ bool useCudfTableHandle(const std::vector>& splitInfo #endif } +bool isDeltaMetadata(const std::unordered_map& metadata) { + auto tableFormatIt = metadata.find(kTableFormatKey); + return tableFormatIt != metadata.end() && tableFormatIt->second == kDeltaTableFormat; +} + +bool isDeltaSplitInfo(const std::shared_ptr& splitInfo) { + if (std::dynamic_pointer_cast(splitInfo) != nullptr) { + return true; + } + for (const auto& metadata : splitInfo->metadataColumns) { + if (isDeltaMetadata(metadata)) { + return true; + } + } + return false; +} + core::SortOrder toSortOrder(const ::substrait::SortField& sortField) { switch (sortField.direction()) { case ::substrait::SortField_SortDirection_SORT_DIRECTION_ASC_NULLS_FIRST: @@ -1573,8 +1595,9 @@ core::PlanNodePtr SubstraitToVeloxPlanConverter::toVeloxPlan(const ::substrait:: connector::ConnectorTableHandlePtr tableHandle; auto remainingFilter = readRel.has_filter() ? exprConverter_->toVeloxExpr(readRel.filter(), baseSchema) : nullptr; - auto connectorId = connectorIds_.hive; - if (useCudfTableHandle(splitInfos_) && veloxCfg_->get(kCudfEnableTableScan, kCudfEnableTableScanDefault) && + auto connectorId = isDeltaSplitInfo(splitInfo) ? connectorIds_.delta : connectorIds_.hive; + if (connectorId == connectorIds_.hive && useCudfTableHandle(splitInfos_) && + veloxCfg_->get(kCudfEnableTableScan, kCudfEnableTableScanDefault) && veloxCfg_->get(kCudfEnabled, kCudfEnabledDefault)) { #ifdef GLUTEN_ENABLE_GPU connectorId = connectorIds_.cudfHive; diff --git a/gluten-delta/src/main/scala/org/apache/gluten/execution/DeltaScanTransformer.scala b/gluten-delta/src/main/scala/org/apache/gluten/execution/DeltaScanTransformer.scala index 1be03dd404a..5f2a5300130 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/execution/DeltaScanTransformer.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/execution/DeltaScanTransformer.scala @@ -55,18 +55,6 @@ case class DeltaScanTransformer( override lazy val fileFormat: ReadFileFormat = ReadFileFormat.ParquetReadFormat - override protected def doValidateInternal(): ValidationResult = { - if ( - requiredSchema.fields.exists( - _.name == "__delta_internal_is_row_deleted") || requiredSchema.fields.exists( - _.name == "__delta_internal_row_index") - ) { - return ValidationResult.failed(s"Deletion vector is not supported in native.") - } - - super.doValidateInternal() - } - override def doCanonicalize(): DeltaScanTransformer = { DeltaScanTransformer( relation, @@ -90,7 +78,6 @@ case class DeltaScanTransformer( } object DeltaScanTransformer { - def apply(scanExec: FileSourceScanExec): DeltaScanTransformer = { new DeltaScanTransformer( scanExec.relation, diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala index e16a6d12fda..f6a414db0f3 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala @@ -16,28 +16,37 @@ */ package org.apache.gluten.extension -import org.apache.gluten.execution.{DeltaScanTransformer, ProjectExecTransformer} +import org.apache.gluten.backendsapi.BackendsApiManager +import org.apache.gluten.execution.{DeltaScanTransformer, FilterExecTransformerBase, ProjectExecTransformer} import org.apache.gluten.extension.columnar.transition.RemoveTransitions import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, CreateNamedStruct, Expression, GetStructField, If, InputFileBlockLength, InputFileBlockStart, InputFileName, IsNull, LambdaFunction, Literal, NamedLambdaVariable} -import org.apache.spark.sql.catalyst.expressions.{ArrayTransform, TransformKeys, TransformValues} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, AttributeReference, Expression, InputFileBlockLength, InputFileBlockStart, InputFileName, NamedExpression} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaParquetFileFormat, NoMapping} -import org.apache.spark.sql.execution.{ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType} +import org.apache.spark.sql.types.StructType -import scala.collection.mutable import scala.collection.mutable.ListBuffer object DeltaPostTransformRules { def rules: Seq[Rule[SparkPlan]] = - RemoveTransitions :: pushDownInputFileExprRule :: columnMappingRule :: Nil + RemoveTransitions :: + nativeDeletionVectorRule :: + pushDownInputFileExprRule :: + columnMappingRule :: Nil + + private val deletionVectorDeletedRowColumnName = "__delta_internal_is_row_deleted" + private val deletionVectorRowIndexColumnName = "__delta_internal_row_index" + private val deletionVectorInternalColumnNames = + Set(deletionVectorDeletedRowColumnName, deletionVectorRowIndexColumnName) private val COLUMN_MAPPING_RULE_TAG: TreeNodeTag[String] = TreeNodeTag[String]("org.apache.gluten.delta.column.mapping") + private val PRESERVE_DELETION_VECTOR_ROW_INDEX_TAG: TreeNodeTag[Boolean] = + TreeNodeTag[Boolean]("org.apache.gluten.delta.preserve.deletion.vector.row.index") private def notAppliedColumnMappingRule(plan: SparkPlan): Boolean = { plan.getTagValue(COLUMN_MAPPING_RULE_TAG).isEmpty @@ -65,6 +74,87 @@ object DeltaPostTransformRules { child.copy(output = p.output) } + /** + * Spark Delta injects synthetic deletion-vector predicates and columns into the plan. Those are + * needed for the JVM reader path, but for the native Delta scan path they must be stripped or + * they will be applied twice with incompatible semantics. + */ + val nativeDeletionVectorRule: Rule[SparkPlan] = (plan: SparkPlan) => { + tagRowIndexRequiredSubtrees(plan) + plan.transformUp { + case scan: DeltaScanTransformer => + val cleanedDataFilters = scan.dataFilters.flatMap(stripDeletionVectorPredicate) + val cleanedPushDownFilters = + scan.pushDownFilters.map(_.flatMap(stripDeletionVectorPredicate)) + val preserveRowIndex = shouldPreserveDeletionVectorRowIndex(scan) + val cleanedOutput = stripDeletionVectorInternalOutput(scan.output, preserveRowIndex) + val cleanedRequiredSchema = + stripDeletionVectorInternalSchema(scan.requiredSchema, preserveRowIndex) + if ( + cleanedDataFilters == scan.dataFilters && + cleanedPushDownFilters == scan.pushDownFilters && + cleanedOutput == scan.output && + cleanedRequiredSchema == scan.requiredSchema + ) { + scan + } else { + scan.copy( + output = cleanedOutput, + requiredSchema = cleanedRequiredSchema, + dataFilters = cleanedDataFilters, + pushDownFilters = cleanedPushDownFilters) + } + case project: ProjectExecTransformer if containsNativeDeltaScan(project.child) => + val cleanedProjectList = stripDeletionVectorInternalProjectList( + project.projectList, + shouldPreserveDeletionVectorRowIndex(project)) + if (cleanedProjectList == project.projectList) { + project + } else if (cleanedProjectList.isEmpty) { + project.child + } else { + ProjectExecTransformer(cleanedProjectList, project.child) + } + case project: ProjectExec if containsNativeDeltaScan(project.child) => + val cleanedProjectList = stripDeletionVectorInternalProjectList( + project.projectList, + shouldPreserveDeletionVectorRowIndex(project)) + if (cleanedProjectList == project.projectList) { + project + } else if (cleanedProjectList.isEmpty) { + project.child + } else { + ProjectExec(cleanedProjectList, project.child) + } + case filter: FilterExecTransformerBase if containsNativeDeltaScan(filter.child) => + stripDeletionVectorPredicate(filter.cond) match { + case Some(cleanCondition) if cleanCondition != filter.cond => + BackendsApiManager.getSparkPlanExecApiInstance + .genFilterExecTransformer(cleanCondition, filter.child) + case Some(_) => + filter + case None => + filter.child + } + case filter: FilterExec if containsNativeDeltaScan(filter.child) => + stripDeletionVectorPredicate(filter.condition) match { + case Some(cleanCondition) if cleanCondition != filter.condition => + FilterExec(cleanCondition, filter.child) + case Some(_) => + filter + case None => + filter.child + } + } + } + + private def containsNativeDeltaScan(plan: SparkPlan): Boolean = { + plan.exists { + case _: DeltaScanTransformer => true + case _ => false + } + } + private def isDeltaColumnMappingFileFormat(fileFormat: FileFormat): Boolean = fileFormat match { case d: DeltaParquetFileFormat if d.columnMappingMode != NoMapping => true @@ -79,6 +169,82 @@ object DeltaPostTransformRules { } } + private def referencesDeletionVectorInternalColumn(expr: Expression): Boolean = { + expr.references.exists(attr => deletionVectorInternalColumnNames.contains(attr.name)) + } + + private def referencesDeletionVectorRowIndex(expr: Expression): Boolean = { + expr.references.exists(_.name == deletionVectorRowIndexColumnName) + } + + private def tagRowIndexRequiredSubtrees(plan: SparkPlan): Unit = { + def tagSubtree(subtree: SparkPlan): Unit = { + subtree.foreach(_.setTagValue(PRESERVE_DELETION_VECTOR_ROW_INDEX_TAG, true)) + } + + def visit(node: SparkPlan): Unit = { + val shouldPreserveRowIndex = + node.expressions.exists(containsIncrementMetricExpr) || + node.expressions.exists(referencesDeletionVectorRowIndex) + if (shouldPreserveRowIndex) { + node.children.foreach(tagSubtree) + } + node.children.foreach(visit) + } + + visit(plan) + } + + private def shouldPreserveDeletionVectorRowIndex(plan: SparkPlan): Boolean = { + plan.getTagValue(PRESERVE_DELETION_VECTOR_ROW_INDEX_TAG).contains(true) || + plan.expressions.exists(containsIncrementMetricExpr) || + plan.expressions.exists(referencesDeletionVectorRowIndex) + } + + private def shouldStripDeletionVectorInternalColumn( + columnName: String, + preserveRowIndex: Boolean): Boolean = { + columnName == deletionVectorDeletedRowColumnName || + (!preserveRowIndex && columnName == deletionVectorRowIndexColumnName) + } + + private def stripDeletionVectorInternalOutput( + output: Seq[Attribute], + preserveRowIndex: Boolean): Seq[Attribute] = { + output.filterNot(attr => shouldStripDeletionVectorInternalColumn(attr.name, preserveRowIndex)) + } + + private def stripDeletionVectorInternalProjectList( + projectList: Seq[NamedExpression], + preserveRowIndex: Boolean): Seq[NamedExpression] = { + projectList.filterNot( + expr => shouldStripDeletionVectorInternalColumn(expr.name, preserveRowIndex)) + } + + private def stripDeletionVectorInternalSchema( + schema: StructType, + preserveRowIndex: Boolean): StructType = { + StructType( + schema.filterNot( + field => shouldStripDeletionVectorInternalColumn(field.name, preserveRowIndex))) + } + + private def stripDeletionVectorPredicate(expr: Expression): Option[Expression] = { + expr match { + case And(left, right) => + (stripDeletionVectorPredicate(left), stripDeletionVectorPredicate(right)) match { + case (Some(cleanLeft), Some(cleanRight)) => Some(And(cleanLeft, cleanRight)) + case (Some(cleanLeft), None) => Some(cleanLeft) + case (None, Some(cleanRight)) => Some(cleanRight) + case (None, None) => None + } + case other if referencesDeletionVectorInternalColumn(other) => + None + case other => + Some(other) + } + } + private def isInputFileRelatedAttribute(attr: Attribute): Boolean = { attr match { case AttributeReference(name, _, _, _) => @@ -96,73 +262,6 @@ object DeltaPostTransformRules { } } - /** - * Checks whether two structurally compatible DataTypes have different struct field names at any - * nesting level. - */ - private def nestedFieldNamesDiffer(logical: DataType, physical: DataType): Boolean = { - (logical, physical) match { - case (l: StructType, p: StructType) if l.length == p.length => - l.zip(p).exists { - case (lf, pf) => - lf.name != pf.name || nestedFieldNamesDiffer(lf.dataType, pf.dataType) - } - case (l: ArrayType, p: ArrayType) => - nestedFieldNamesDiffer(l.elementType, p.elementType) - case (l: MapType, p: MapType) => - nestedFieldNamesDiffer(l.keyType, p.keyType) || - nestedFieldNamesDiffer(l.valueType, p.valueType) - case _ => false - } - } - - /** - * Rebuilds an expression tree so that nested struct field names match the logical schema. Uses - * positional extraction (GetStructField) and reconstruction (CreateNamedStruct) instead of Cast, - * so correctness does not depend on Velox's cast_match_struct_by_name config. - */ - private def reconcileFieldNames( - expr: Expression, - logical: DataType, - physical: DataType): Expression = { - (logical, physical) match { - case (l: StructType, p: StructType) if l.length == p.length => - val rebuiltFields = l.zip(p).zipWithIndex.flatMap { - case ((lf, pf), i) => - val extracted = GetStructField(expr, i, None) - val reconciled = reconcileFieldNames(extracted, lf.dataType, pf.dataType) - Seq(Literal(lf.name), reconciled) - } - val rebuilt = CreateNamedStruct(rebuiltFields) - If(IsNull(expr), Literal.create(null, l), rebuilt) - case (l: ArrayType, p: ArrayType) if nestedFieldNamesDiffer(l.elementType, p.elementType) => - val lambdaVar = NamedLambdaVariable("element", p.elementType, p.containsNull) - val body = reconcileFieldNames(lambdaVar, l.elementType, p.elementType) - ArrayTransform(expr, LambdaFunction(body, Seq(lambdaVar))) - case (l: MapType, p: MapType) => - val needKeys = nestedFieldNamesDiffer(l.keyType, p.keyType) - val needValues = nestedFieldNamesDiffer(l.valueType, p.valueType) - var result = expr - if (needValues) { - val keyVar = NamedLambdaVariable("key", p.keyType, false) - val valueVar = NamedLambdaVariable("value", p.valueType, p.valueContainsNull) - val body = reconcileFieldNames(valueVar, l.valueType, p.valueType) - result = TransformValues(result, LambdaFunction(body, Seq(keyVar, valueVar))) - } - if (needKeys) { - val keyVar = NamedLambdaVariable("key", p.keyType, false) - val valueVar = NamedLambdaVariable( - "value", - if (needValues) l.valueType else p.valueType, - p.valueContainsNull) - val body = reconcileFieldNames(keyVar, l.keyType, p.keyType) - result = TransformKeys(result, LambdaFunction(body, Seq(keyVar, valueVar))) - } - result - case _ => expr - } - } - /** * This method is only used for Delta ColumnMapping FileFormat(e.g. nameMapping and idMapping) * transform the metadata of Delta into Parquet's, each plan should only be transformed once. @@ -185,9 +284,8 @@ object DeltaPostTransformRules { )(SparkSession.active) // transform output's name into physical name so Reader can read data correctly // should keep the columns order the same as the origin output - case class ColumnMapping(logicalName: String, logicalType: DataType, physicalAttr: Attribute) - val columnMappings = ListBuffer.empty[ColumnMapping] - val seenNames = mutable.Set.empty[String] + val originColumnNames = ListBuffer.empty[String] + val transformedAttrs = ListBuffer.empty[Attribute] def mapAttribute(attr: Attribute) = { val newAttr = if (plan.isMetadataColumn(attr)) { attr @@ -198,8 +296,9 @@ object DeltaPostTransformRules { .createPhysicalAttributes(Seq(attr), fmt.referenceSchema, fmt.columnMappingMode) .head } - if (seenNames.add(attr.name)) { - columnMappings += ColumnMapping(attr.name, attr.dataType, newAttr) + if (!originColumnNames.contains(attr.name)) { + transformedAttrs += newAttr + originColumnNames += attr.name } newAttr } @@ -239,20 +338,9 @@ object DeltaPostTransformRules { scanExecTransformer.copyTagsFrom(plan) tagColumnMappingRule(scanExecTransformer) - // Alias physical names back to logical names. For struct-typed columns, Delta column - // mapping renames internal field names to physical UUIDs. A top-level Alias only restores - // the column name, not the struct's internal field names. We rebuild the struct with - // logical field names using positional extraction (GetStructField/CreateNamedStruct) - // instead of Cast, so correctness does not depend on any Velox cast config. - val expr = columnMappings.map { - cm => - val projectedExpr: Expression = - if (nestedFieldNamesDiffer(cm.logicalType, cm.physicalAttr.dataType)) { - reconcileFieldNames(cm.physicalAttr, cm.logicalType, cm.physicalAttr.dataType) - } else { - cm.physicalAttr - } - Alias(projectedExpr, cm.logicalName)(exprId = cm.physicalAttr.exprId) + // alias physicalName into tableName + val expr = (transformedAttrs, originColumnNames).zipped.map { + (attr, columnName) => Alias(attr, columnName)(exprId = attr.exprId) } val projectExecTransformer = ProjectExecTransformer(expr.toSeq, scanExecTransformer) projectExecTransformer diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala index 5fe1b4ba86e..ebafb0c08c3 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala @@ -17,16 +17,93 @@ package org.apache.gluten.extension import org.apache.gluten.execution.DeltaScanTransformer +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.extension.columnar.offload.OffloadSingleNode import org.apache.spark.sql.delta.DeltaParquetFileFormat +import org.apache.spark.sql.delta.SnapshotDescriptor +import org.apache.spark.sql.delta.commands.DeletionVectorUtils.deletionVectorsReadable +import org.apache.spark.sql.delta.files.TahoeFileIndex +import org.apache.spark.sql.delta.stats.PreparedDeltaFileIndex import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.util.SparkVersionUtil case class OffloadDeltaScan() extends OffloadSingleNode { + private val DeletionVectorsUseMetadataRowIndexKey = + "spark.databricks.delta.deletionVectors.useMetadataRowIndex" + override def offload(plan: SparkPlan): SparkPlan = plan match { + case scan: FileSourceScanExec if isDeltaLogScan(scan) => + FallbackTags.add(scan, "fallback Delta _delta_log scan") + scan + case scan: FileSourceScanExec if shouldFallbackSpark34DeletionVectorScan(scan) => + FallbackTags.add(scan, "fallback Spark 3.4 Delta DV scan") + scan case scan: FileSourceScanExec - if scan.relation.fileFormat.getClass == classOf[DeltaParquetFileFormat] => + if shouldFallbackDeletionVectorScanWithoutMetadataRowIndex(scan) => + FallbackTags.add(scan, "fallback Delta DV scan without metadata row index") + scan + case scan: FileSourceScanExec if isDeltaScan(scan) => DeltaScanTransformer(scan) case other => other } + + private def isDeltaScan(scan: FileSourceScanExec): Boolean = { + isDeltaFileIndex(scan) || isDeltaParquetScan(scan) + } + + private def isDeltaParquetScan(scan: FileSourceScanExec): Boolean = { + val fileFormatClass = scan.relation.fileFormat.getClass + fileFormatClass == classOf[DeltaParquetFileFormat] || + fileFormatClass.getSimpleName == "GlutenDeltaParquetFileFormat" + } + + private def isDeltaFileIndex(scan: FileSourceScanExec): Boolean = { + scan.relation.location.isInstanceOf[TahoeFileIndex] || + scan.relation.location.isInstanceOf[PreparedDeltaFileIndex] + } + + private def isDeltaLogScan(scan: FileSourceScanExec): Boolean = { + scan.relation.location.rootPaths.exists { + path => + val root = path.toString + root.contains("/_delta_log") || root.contains("\\_delta_log") || root.endsWith("_delta_log") + } + } + + private def shouldFallbackSpark34DeletionVectorScan(scan: FileSourceScanExec): Boolean = { + if (SparkVersionUtil.gteSpark35) { + return false + } + + containsDeletionVector(scan) + } + + private def shouldFallbackDeletionVectorScanWithoutMetadataRowIndex( + scan: FileSourceScanExec): Boolean = { + if (!SparkVersionUtil.gteSpark35) { + return false + } + + // Delta DML tests force this path and rely on Spark's injected + // row-index filter column for correctness. Keep it on Spark until the native path can + // prove the same contract for DML-generated DVs. + val useMetadataRowIndex = + scan.relation.sparkSession.sessionState.conf + .getConfString(DeletionVectorsUseMetadataRowIndexKey, "true") + .toBoolean + !useMetadataRowIndex && containsDeletionVector(scan) + } + + private def containsDeletionVector(scan: FileSourceScanExec): Boolean = { + scan.relation.location match { + case preparedIndex: PreparedDeltaFileIndex => + preparedIndex.preparedScan.files.exists(_.deletionVector != null) + case index: TahoeFileIndex => + val snapshot = index.asInstanceOf[SnapshotDescriptor] + deletionVectorsReadable(snapshot.protocol, snapshot.metadata) + case _ => + false + } + } } diff --git a/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala b/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala index 031bf460347..d7f9e66e579 100644 --- a/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala +++ b/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala @@ -19,6 +19,7 @@ package org.apache.gluten.execution import org.apache.spark.SparkConf import org.apache.spark.sql.Row import org.apache.spark.sql.types._ +import org.apache.spark.util.SparkVersionUtil import scala.collection.JavaConverters._ @@ -37,6 +38,7 @@ abstract class DeltaSuite extends WholeStageTransformerSuite { .set("spark.memory.offHeap.size", "2g") .set("spark.unsafe.exceptionOnMemoryLeak", "true") .set("spark.sql.autoBroadcastJoinThreshold", "-1") + .set("spark.sql.ansi.enabled", "false") .set("spark.sql.sources.useV1SourceList", "avro") .set("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") @@ -209,12 +211,16 @@ abstract class DeltaSuite extends WholeStageTransformerSuite { s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") checkAnswer(spark.read.format("delta").load(path), df1.union(df2)) spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (${values2.mkString(", ")})") - import org.apache.spark.sql.execution.GlutenImplicits._ val df = spark.read.format("delta").load(path) - assert( - df.fallbackSummary.fallbackNodeToReason - .flatMap(_.values) - .exists(_.contains("Deletion vector is not supported in native"))) + val executedPlan = df.queryExecution.executedPlan + if (SparkVersionUtil.gteSpark35) { + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.nonEmpty) + val planText = executedPlan.toString() + assert(!planText.contains("__delta_internal_is_row_deleted")) + assert(!planText.contains("__delta_internal_row_index")) + } else { + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.isEmpty) + } checkAnswer(df, df1) } } @@ -320,13 +326,13 @@ abstract class DeltaSuite extends WholeStageTransformerSuite { withSQLConf("spark.gluten.sql.columnar.scanOnly" -> "true") { withTable("delta_pf") { spark.sql(s""" - |create table test (id int, name string) using delta + |create table delta_pf (id int, name string) using delta |""".stripMargin) spark.sql(s""" - |insert into test values (1, "v1"), (2, "v2"), (3, "v1"), (4, "v2") + |insert into delta_pf values (1, "v1"), (2, "v2"), (3, "v1"), (4, "v2") |""".stripMargin) runQueryAndCompare( - "select id from test where name > 'v1'", + "select id from delta_pf where name > 'v1'", compareResult = true, noFallBack = false) { df => diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/DeltaLocalFilesBuilder.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/DeltaLocalFilesBuilder.java new file mode 100644 index 00000000000..fc75285eddb --- /dev/null +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/DeltaLocalFilesBuilder.java @@ -0,0 +1,56 @@ +/* + * 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.substrait.rel; + +import org.apache.gluten.substrait.rel.DeltaLocalFilesNode.DeltaFileReadOptions; + +import java.util.List; +import java.util.Map; + +public class DeltaLocalFilesBuilder { + private DeltaLocalFilesBuilder() {} + + public static DeltaLocalFilesNode makeDeltaLocalFiles( + Integer index, + List paths, + List starts, + List lengths, + List fileSizes, + List modificationTimes, + List> partitionColumns, + List> metadataColumns, + LocalFilesNode.ReadFileFormat fileFormat, + List preferredLocations, + Map properties, + List> otherMetadataColumns, + List deltaReadOptions) { + return new DeltaLocalFilesNode( + index, + paths, + starts, + lengths, + fileSizes, + modificationTimes, + partitionColumns, + metadataColumns, + fileFormat, + preferredLocations, + properties, + otherMetadataColumns, + deltaReadOptions); + } +} diff --git a/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/DeltaLocalFilesNode.java b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/DeltaLocalFilesNode.java new file mode 100644 index 00000000000..dd34838261c --- /dev/null +++ b/gluten-substrait/src/main/java/org/apache/gluten/substrait/rel/DeltaLocalFilesNode.java @@ -0,0 +1,132 @@ +/* + * 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.substrait.rel; + +import com.google.protobuf.ByteString; +import io.substrait.proto.ReadRel; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +public class DeltaLocalFilesNode extends LocalFilesNode { + private final List deltaReadOptions = new ArrayList<>(); + + DeltaLocalFilesNode( + Integer index, + List paths, + List starts, + List lengths, + List fileSizes, + List modificationTimes, + List> partitionColumns, + List> metadataColumns, + ReadFileFormat fileFormat, + List preferredLocations, + Map properties, + List> otherMetadataColumns, + List deltaReadOptions) { + super( + index, + paths, + starts, + lengths, + fileSizes, + modificationTimes, + partitionColumns, + metadataColumns, + fileFormat, + preferredLocations, + properties, + otherMetadataColumns); + this.deltaReadOptions.addAll(deltaReadOptions); + } + + @Override + protected void processFileBuilder(ReadRel.LocalFiles.FileOrFiles.Builder fileBuilder, int index) { + DeltaFileReadOptions options = deltaReadOptions.get(index); + ReadRel.LocalFiles.FileOrFiles.DeltaReadOptions.Builder deltaBuilder = + ReadRel.LocalFiles.FileOrFiles.DeltaReadOptions.newBuilder() + .setRowIndexFilterType(toProtoRowIndexFilterType(options.rowIndexFilterType())) + .setHasDeletionVector(options.hasDeletionVector()); + + if (options.hasDeletionVector()) { + deltaBuilder + .setDeletionVectorCardinality(options.deletionVectorCardinality()) + .setSerializedDeletionVector(ByteString.copyFrom(options.serializedDeletionVector())); + } + + fileBuilder.setDelta(deltaBuilder.build()); + } + + private static ReadRel.LocalFiles.FileOrFiles.DeltaReadOptions.RowIndexFilterType + toProtoRowIndexFilterType(RowIndexFilterType rowIndexFilterType) { + switch (rowIndexFilterType) { + case IF_CONTAINED: + return ReadRel.LocalFiles.FileOrFiles.DeltaReadOptions.RowIndexFilterType.IF_CONTAINED; + case IF_NOT_CONTAINED: + return ReadRel.LocalFiles.FileOrFiles.DeltaReadOptions.RowIndexFilterType.IF_NOT_CONTAINED; + case KEEP_ALL: + default: + return ReadRel.LocalFiles.FileOrFiles.DeltaReadOptions.RowIndexFilterType.KEEP_ALL; + } + } + + public enum RowIndexFilterType { + KEEP_ALL, + IF_CONTAINED, + IF_NOT_CONTAINED + } + + public static class DeltaFileReadOptions implements Serializable { + private static final long serialVersionUID = 1L; + + private final RowIndexFilterType rowIndexFilterType; + private final boolean hasDeletionVector; + private final long deletionVectorCardinality; + private final byte[] serializedDeletionVector; + + public DeltaFileReadOptions( + RowIndexFilterType rowIndexFilterType, + boolean hasDeletionVector, + long deletionVectorCardinality, + byte[] serializedDeletionVector) { + this.rowIndexFilterType = rowIndexFilterType; + this.hasDeletionVector = hasDeletionVector; + this.deletionVectorCardinality = deletionVectorCardinality; + this.serializedDeletionVector = + serializedDeletionVector == null ? new byte[0] : serializedDeletionVector; + } + + public RowIndexFilterType rowIndexFilterType() { + return rowIndexFilterType; + } + + public boolean hasDeletionVector() { + return hasDeletionVector; + } + + public long deletionVectorCardinality() { + return deletionVectorCardinality; + } + + public byte[] serializedDeletionVector() { + return serializedDeletionVector; + } + } +} diff --git a/gluten-substrait/src/main/resources/substrait/proto/substrait/algebra.proto b/gluten-substrait/src/main/resources/substrait/proto/substrait/algebra.proto index 2bfb68e0979..02c7f4cc5c6 100644 --- a/gluten-substrait/src/main/resources/substrait/proto/substrait/algebra.proto +++ b/gluten-substrait/src/main/resources/substrait/proto/substrait/algebra.proto @@ -197,6 +197,18 @@ message ReadRel { repeated DeleteFile delete_files = 3; } + message DeltaReadOptions { + enum RowIndexFilterType { + KEEP_ALL = 0; + IF_CONTAINED = 1; + IF_NOT_CONTAINED = 2; + } + RowIndexFilterType row_index_filter_type = 1; + bool has_deletion_vector = 2; + uint64 deletion_vector_cardinality = 3; + bytes serialized_deletion_vector = 4; + } + // File reading options oneof file_format { ParquetReadOptions parquet = 9; @@ -207,6 +219,7 @@ message ReadRel { TextReadOptions text = 14; JsonReadOptions json = 15; IcebergReadOptions iceberg = 16; + DeltaReadOptions delta = 22; } message partitionColumn { From 32ba5acfa7efc22c91715c8da090d84b20c63759 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 14:03:29 +0300 Subject: [PATCH 03/16] [VL][Delta] Guard DV DML row-index scans Keep Delta DV DML row-index target scans on Spark unless native DML row-index scanning and native write are explicitly enabled. Preserve the Spark Project/Filter subtree above the fallback scan and add Delta 3.3/4.0 plan-shape coverage for metadata row-index on and off. Validation: JAVA_HOME=/opt/homebrew/opt/openjdk@17/libexec/openjdk.jdk/Contents/Home ./build/mvn test-compile -pl backends-velox -am -Pjava-17,spark-3.5,backends-velox,hadoop-3.3,spark-ut,delta -DskipTests Validation: JAVA_HOME=/opt/homebrew/opt/openjdk@17/libexec/openjdk.jdk/Contents/Home ./build/mvn test-compile -pl backends-velox -am -Pjava-17,spark-4.0,scala-2.13,backends-velox,hadoop-3.3,spark-ut,delta -DskipTests Validation: git diff --cached --check --- .../DeltaDeletionVectorHandoffSuite.scala | 77 ++++++++++++++++++- .../DeltaDeletionVectorHandoffSuite.scala | 76 +++++++++++++++++- .../DeltaDeletionVectorDmlUtils.scala | 57 ++++++++++++++ .../extension/DeltaPostTransformRules.scala | 59 +++++++++++++- .../gluten/extension/OffloadDeltaScan.scala | 36 ++++++--- 5 files changed, 289 insertions(+), 16 deletions(-) create mode 100644 gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala index f5510a95255..7f5ddd923b4 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -16,12 +16,17 @@ */ package org.apache.spark.sql.delta -import org.apache.gluten.execution.DeltaScanTransformer +import org.apache.gluten.execution.{DeltaScanTransformer, FilterExecTransformerBase, ProjectExecTransformerBase} +import org.apache.gluten.extension.DeltaDeletionVectorDmlUtils +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.{DeltaSQLCommandTest, DeltaSQLTestUtils} +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.tags.ExtendedSQLTest +import org.apache.spark.util.SparkVersionUtil import org.apache.hadoop.fs.Path @@ -34,6 +39,33 @@ class DeltaDeletionVectorHandoffSuite import testImplicits._ + private val DmlFallbackReason = "fallback Delta DV DML row-index scan" + + private def containsDmlFallbackScan(plan: SparkPlan): Boolean = { + plan.exists { + case scan: FileSourceScanExec => + DeltaDeletionVectorDmlUtils.isDeletionVectorDmlRowIndexScan(scan) && + FallbackTags.getOption(scan).exists(_.reason().contains(DmlFallbackReason)) + case _ => false + } + } + + private def hasSparkParentOverDmlFallbackScan(plan: SparkPlan): Boolean = { + plan.exists { + case ProjectExec(_, child) if containsDmlFallbackScan(child) => true + case FilterExec(_, child) if containsDmlFallbackScan(child) => true + case _ => false + } + } + + private def hasNativeParentOverDmlFallbackScan(plan: SparkPlan): Boolean = { + plan.exists { + case project: ProjectExecTransformerBase if containsDmlFallbackScan(project.child) => true + case filter: FilterExecTransformerBase if containsDmlFallbackScan(filter.child) => true + case _ => false + } + } + test("Spark 3.5 Delta DV scan handoff should filter deleted rows") { withTempDir { tempDir => @@ -65,4 +97,47 @@ class DeltaDeletionVectorHandoffSuite checkAnswer(df, Seq((1, "a"), (2, "b")).toDF()) } } + + Seq(true, false).foreach { + useMetadataRowIndex => + test( + "Delta DV DML row-index scan should fall back with Spark project/filter, " + + s"metadata row index=$useMetadataRowIndex") { + assume(SparkVersionUtil.gteSpark35, "DML row-index scan fallback is Spark 3.5+ coverage") + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES " + + "('delta.enableDeletionVectors' = true)") + + var executedPlans: Seq[SparkPlan] = Seq.empty + withSQLConf( + DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> + useMetadataRowIndex.toString, + "spark.gluten.sql.columnar.backend.velox.delta.enableNativeWrite" -> "false", + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" -> "false" + ) { + executedPlans = DeltaTestUtils.withAllPlansCaptured(spark) { + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)").collect() + }.map(_.executedPlan) + } + val planText = executedPlans.map(_.treeString).mkString("\n\n") + assert(executedPlans.exists(containsDmlFallbackScan), planText) + assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) + assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + + val log = DeltaLog.forTable(spark, new Path(path)) + assert(log.update().allFiles.collect().exists(_.deletionVector != null)) + checkAnswer(spark.read.format("delta").load(path), Seq((1, "a"), (2, "b")).toDF()) + } + } + } } diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala index dda547b015f..e8781816a01 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -16,13 +16,17 @@ */ package org.apache.spark.sql.delta -import org.apache.gluten.execution.DeltaScanTransformer +import org.apache.gluten.execution.{DeltaScanTransformer, FilterExecTransformerBase, ProjectExecTransformerBase} +import org.apache.gluten.extension.DeltaDeletionVectorDmlUtils +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.spark.sql.QueryTest import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.{DeltaSQLCommandTest, DeltaSQLTestUtils} +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.tags.ExtendedSQLTest +import org.apache.spark.util.SparkVersionUtil import org.apache.hadoop.fs.Path @@ -35,6 +39,33 @@ class DeltaDeletionVectorHandoffSuite import testImplicits._ + private val DmlFallbackReason = "fallback Delta DV DML row-index scan" + + private def containsDmlFallbackScan(plan: SparkPlan): Boolean = { + plan.exists { + case scan: FileSourceScanExec => + DeltaDeletionVectorDmlUtils.isDeletionVectorDmlRowIndexScan(scan) && + FallbackTags.getOption(scan).exists(_.reason().contains(DmlFallbackReason)) + case _ => false + } + } + + private def hasSparkParentOverDmlFallbackScan(plan: SparkPlan): Boolean = { + plan.exists { + case ProjectExec(_, child) if containsDmlFallbackScan(child) => true + case FilterExec(_, child) if containsDmlFallbackScan(child) => true + case _ => false + } + } + + private def hasNativeParentOverDmlFallbackScan(plan: SparkPlan): Boolean = { + plan.exists { + case project: ProjectExecTransformerBase if containsDmlFallbackScan(project.child) => true + case filter: FilterExecTransformerBase if containsDmlFallbackScan(filter.child) => true + case _ => false + } + } + test("Spark 4 Delta DV scan should fall back when metadata row index is disabled") { withTempDir { tempDir => @@ -64,6 +95,49 @@ class DeltaDeletionVectorHandoffSuite } } + Seq(true, false).foreach { + useMetadataRowIndex => + test( + "Delta DV DML row-index scan should fall back with Spark project/filter, " + + s"metadata row index=$useMetadataRowIndex") { + assume(SparkVersionUtil.gteSpark35, "DML row-index scan fallback is Spark 3.5+ coverage") + withTempDir { + tempDir => + val path = tempDir.getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES " + + "('delta.enableDeletionVectors' = true)") + + var executedPlans: Seq[SparkPlan] = Seq.empty + withSQLConf( + DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> + useMetadataRowIndex.toString, + "spark.gluten.sql.columnar.backend.velox.delta.enableNativeWrite" -> "false", + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" -> "false" + ) { + executedPlans = DeltaTestUtils.withAllPlansCaptured(spark) { + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)").collect() + }.map(_.executedPlan) + } + val planText = executedPlans.map(_.treeString).mkString("\n\n") + assert(executedPlans.exists(containsDmlFallbackScan), planText) + assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) + assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + + val log = DeltaLog.forTable(spark, new Path(path)) + assert(log.update().allFiles.collect().exists(_.deletionVector != null)) + checkAnswer(spark.read.format("delta").load(path), Seq((1, "a"), (2, "b")).toDF()) + } + } + } + test("Spark 4 Delta DV scan handoff should filter deleted rows") { withTempDir { tempDir => diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala new file mode 100644 index 00000000000..a61dac37a36 --- /dev/null +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala @@ -0,0 +1,57 @@ +/* + * 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.extension + +import org.apache.spark.sql.delta.DeltaParquetFileFormat +import org.apache.spark.sql.delta.files.TahoeFileIndex +import org.apache.spark.sql.delta.stats.PreparedDeltaFileIndex +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat + +object DeltaDeletionVectorDmlUtils { + private val deletionVectorRowIndexColumnNames = + Set( + "__delta_internal_row_index", + DeltaParquetFileFormat.ROW_INDEX_COLUMN_NAME, + ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME) + private val filePathColumnNames = Set("file_path", "filePath") + + def isDeltaScan(scan: FileSourceScanExec): Boolean = { + isDeltaFileIndex(scan) || isDeltaParquetScan(scan) + } + + def isDeltaParquetScan(scan: FileSourceScanExec): Boolean = { + val fileFormatClass = scan.relation.fileFormat.getClass + fileFormatClass == classOf[DeltaParquetFileFormat] || + fileFormatClass.getSimpleName == "GlutenDeltaParquetFileFormat" + } + + def isDeltaFileIndex(scan: FileSourceScanExec): Boolean = { + scan.relation.location.isInstanceOf[TahoeFileIndex] || + scan.relation.location.isInstanceOf[PreparedDeltaFileIndex] + } + + def isDeletionVectorDmlRowIndexScan(scan: FileSourceScanExec): Boolean = { + if (!isDeltaScan(scan)) { + return false + } + + val scanColumnNames = (scan.output.map(_.name) ++ scan.requiredSchema.fieldNames).toSet + scanColumnNames.exists(deletionVectorRowIndexColumnNames.contains) && + scanColumnNames.exists(filePathColumnNames.contains) + } +} diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala index f6a414db0f3..501a8a19d4d 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala @@ -17,7 +17,8 @@ package org.apache.gluten.extension import org.apache.gluten.backendsapi.BackendsApiManager -import org.apache.gluten.execution.{DeltaScanTransformer, FilterExecTransformerBase, ProjectExecTransformer} +import org.apache.gluten.execution.{DeltaScanTransformer, FilterExecTransformerBase, ProjectExecTransformer, ProjectExecTransformerBase} +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.extension.columnar.transition.RemoveTransitions import org.apache.spark.sql.SparkSession @@ -25,8 +26,9 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, Attribu import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaParquetFileFormat, NoMapping} -import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} +import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.types.StructType import scala.collection.mutable.ListBuffer @@ -34,14 +36,23 @@ import scala.collection.mutable.ListBuffer object DeltaPostTransformRules { def rules: Seq[Rule[SparkPlan]] = RemoveTransitions :: + keepDmlRowIndexFallbackSubtreeOnSpark :: nativeDeletionVectorRule :: pushDownInputFileExprRule :: columnMappingRule :: Nil private val deletionVectorDeletedRowColumnName = "__delta_internal_is_row_deleted" private val deletionVectorRowIndexColumnName = "__delta_internal_row_index" + private val deletionVectorRowIndexColumnNames = + Set( + deletionVectorRowIndexColumnName, + DeltaParquetFileFormat.ROW_INDEX_COLUMN_NAME, + ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME) private val deletionVectorInternalColumnNames = Set(deletionVectorDeletedRowColumnName, deletionVectorRowIndexColumnName) + private val deletionVectorPredicateColumnNames = + deletionVectorInternalColumnNames ++ deletionVectorRowIndexColumnNames + private val deletionVectorFilePathColumnNames = Set("file_path", "filePath") private val COLUMN_MAPPING_RULE_TAG: TreeNodeTag[String] = TreeNodeTag[String]("org.apache.gluten.delta.column.mapping") @@ -74,6 +85,20 @@ object DeltaPostTransformRules { child.copy(output = p.output) } + /** + * Native DELETE/UPDATE/MERGE DV support can deliberately keep the target row-index scan in Spark. + * In that mode, keeping the parent filter/project native adds Spark-row -> Velox-columnar -> + * Spark-row transitions immediately before Delta's JVM bitmap path. Keep the small scan subtree + * in Spark until the native DML row-index scan is fast enough to own the whole path. + */ + val keepDmlRowIndexFallbackSubtreeOnSpark: Rule[SparkPlan] = (plan: SparkPlan) => + plan.transformUp { + case project: ProjectExecTransformerBase if containsDmlRowIndexFallbackScan(project.child) => + ProjectExec(project.list, project.child) + case filter: FilterExecTransformerBase if containsDmlRowIndexFallbackScan(filter.child) => + FilterExec(filter.cond, filter.child) + } + /** * Spark Delta injects synthetic deletion-vector predicates and columns into the plan. Those are * needed for the JVM reader path, but for the native Delta scan path they must be stripped or @@ -155,6 +180,16 @@ object DeltaPostTransformRules { } } + private def containsDmlRowIndexFallbackScan(plan: SparkPlan): Boolean = { + plan.exists { + case scan: FileSourceScanExec => + FallbackTags + .getOption(scan) + .exists(_.reason().contains("fallback Delta DV DML row-index scan")) + case _ => false + } + } + private def isDeltaColumnMappingFileFormat(fileFormat: FileFormat): Boolean = fileFormat match { case d: DeltaParquetFileFormat if d.columnMappingMode != NoMapping => true @@ -170,11 +205,11 @@ object DeltaPostTransformRules { } private def referencesDeletionVectorInternalColumn(expr: Expression): Boolean = { - expr.references.exists(attr => deletionVectorInternalColumnNames.contains(attr.name)) + expr.references.exists(attr => deletionVectorPredicateColumnNames.contains(attr.name)) } private def referencesDeletionVectorRowIndex(expr: Expression): Boolean = { - expr.references.exists(_.name == deletionVectorRowIndexColumnName) + expr.references.exists(attr => deletionVectorRowIndexColumnNames.contains(attr.name)) } private def tagRowIndexRequiredSubtrees(plan: SparkPlan): Unit = { @@ -196,11 +231,27 @@ object DeltaPostTransformRules { } private def shouldPreserveDeletionVectorRowIndex(plan: SparkPlan): Boolean = { + isDeletionVectorDmlRowIndexScan(plan) || plan.getTagValue(PRESERVE_DELETION_VECTOR_ROW_INDEX_TAG).contains(true) || plan.expressions.exists(containsIncrementMetricExpr) || plan.expressions.exists(referencesDeletionVectorRowIndex) } + private def isDeletionVectorDmlRowIndexScan(plan: SparkPlan): Boolean = { + val scanColumnNames = plan match { + case scan: DeltaScanTransformer => + (scan.output.map(_.name) ++ scan.requiredSchema.fieldNames).toSet + case scan: FileSourceScanExec => + (scan.output.map(_.name) ++ scan.requiredSchema.fieldNames).toSet + case _ => + Set.empty[String] + } + val hasRowIndex = scanColumnNames.exists(deletionVectorRowIndexColumnNames.contains) + val hasFilePath = scanColumnNames.exists(deletionVectorFilePathColumnNames.contains) + val hasDeletedRowMarker = scanColumnNames.contains(deletionVectorDeletedRowColumnName) + hasRowIndex && (hasFilePath || !hasDeletedRowMarker) + } + private def shouldStripDeletionVectorInternalColumn( columnName: String, preserveRowIndex: Boolean): Boolean = { diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala index ebafb0c08c3..25a0252c2b0 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala @@ -20,7 +20,6 @@ import org.apache.gluten.execution.DeltaScanTransformer import org.apache.gluten.extension.columnar.FallbackTags import org.apache.gluten.extension.columnar.offload.OffloadSingleNode -import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.SnapshotDescriptor import org.apache.spark.sql.delta.commands.DeletionVectorUtils.deletionVectorsReadable import org.apache.spark.sql.delta.files.TahoeFileIndex @@ -31,6 +30,10 @@ import org.apache.spark.util.SparkVersionUtil case class OffloadDeltaScan() extends OffloadSingleNode { private val DeletionVectorsUseMetadataRowIndexKey = "spark.databricks.delta.deletionVectors.useMetadataRowIndex" + private val enableNativeDeletionVectorDmlRowIndexScanKey = + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" + private val enableNativeDeltaWriteKey = + "spark.gluten.sql.columnar.backend.velox.delta.enableNativeWrite" override def offload(plan: SparkPlan): SparkPlan = plan match { case scan: FileSourceScanExec if isDeltaLogScan(scan) => @@ -39,6 +42,9 @@ case class OffloadDeltaScan() extends OffloadSingleNode { case scan: FileSourceScanExec if shouldFallbackSpark34DeletionVectorScan(scan) => FallbackTags.add(scan, "fallback Spark 3.4 Delta DV scan") scan + case scan: FileSourceScanExec if shouldFallbackDeletionVectorDmlScan(scan) => + FallbackTags.add(scan, "fallback Delta DV DML row-index scan") + scan case scan: FileSourceScanExec if shouldFallbackDeletionVectorScanWithoutMetadataRowIndex(scan) => FallbackTags.add(scan, "fallback Delta DV scan without metadata row index") @@ -49,18 +55,28 @@ case class OffloadDeltaScan() extends OffloadSingleNode { } private def isDeltaScan(scan: FileSourceScanExec): Boolean = { - isDeltaFileIndex(scan) || isDeltaParquetScan(scan) + DeltaDeletionVectorDmlUtils.isDeltaScan(scan) } - private def isDeltaParquetScan(scan: FileSourceScanExec): Boolean = { - val fileFormatClass = scan.relation.fileFormat.getClass - fileFormatClass == classOf[DeltaParquetFileFormat] || - fileFormatClass.getSimpleName == "GlutenDeltaParquetFileFormat" - } + private def shouldFallbackDeletionVectorDmlScan(scan: FileSourceScanExec): Boolean = { + val enableNativeDeltaWrite = + scan.relation.sparkSession.sessionState.conf + .getConfString(enableNativeDeltaWriteKey, "false") + .toBoolean + val enableNativeDmlRowIndexScan = + scan.relation.sparkSession.sessionState.conf + .getConfString(enableNativeDeletionVectorDmlRowIndexScanKey, "false") + .toBoolean + if (enableNativeDeltaWrite && enableNativeDmlRowIndexScan) { + return false + } - private def isDeltaFileIndex(scan: FileSourceScanExec): Boolean = { - scan.relation.location.isInstanceOf[TahoeFileIndex] || - scan.relation.location.isInstanceOf[PreparedDeltaFileIndex] + // DELETE/UPDATE/MERGE with persistent deletion vectors needs the target scan to expose + // per-file row indexes so Delta can build updated DV bitmaps. Gluten can scan the target + // natively, but the bitmap aggregation/update itself still runs in Spark today. Keeping this + // scan on Spark avoids a native scan plus row transition immediately before Spark's DV + // aggregation path. + DeltaDeletionVectorDmlUtils.isDeletionVectorDmlRowIndexScan(scan) } private def isDeltaLogScan(scan: FileSourceScanExec): Boolean = { From 6c91bc5b41397cbb69cefaff1b872bb02f5aa0b3 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 15:45:34 +0300 Subject: [PATCH 04/16] [VL][Delta] Cover existing DV DML scan fallback --- .../DeltaDeletionVectorHandoffSuite.scala | 59 +++++++++++++++++++ .../DeltaDeletionVectorHandoffSuite.scala | 59 +++++++++++++++++++ 2 files changed, 118 insertions(+) diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala index 7f5ddd923b4..48ddca50451 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -30,6 +30,8 @@ import org.apache.spark.util.SparkVersionUtil import org.apache.hadoop.fs.Path +import java.io.File + @ExtendedSQLTest class DeltaDeletionVectorHandoffSuite extends QueryTest @@ -66,6 +68,37 @@ class DeltaDeletionVectorHandoffSuite } } + private def captureDeletePlans( + path: String, + predicate: String, + useMetadataRowIndex: Boolean): Seq[SparkPlan] = { + var executedPlans: Seq[SparkPlan] = Seq.empty + withSQLConf( + DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> + useMetadataRowIndex.toString, + "spark.gluten.sql.columnar.backend.velox.delta.enableNativeWrite" -> "false", + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" -> "false" + ) { + executedPlans = DeltaTestUtils.withAllPlansCaptured(spark) { + spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + }.map(_.executedPlan) + } + executedPlans + } + + private def assertSparkDmlFallback(executedPlans: Seq[SparkPlan]): Unit = { + val planText = executedPlans.map(_.treeString).mkString("\n\n") + assert(executedPlans.exists(containsDmlFallbackScan), planText) + assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) + assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + } + + private def activeDvCardinality(path: String): Long = { + val log = DeltaLog.forTable(spark, new Path(path)) + log.update().allFiles.collect().flatMap( + file => Option(file.deletionVector).map(_.cardinality)).sum + } + test("Spark 3.5 Delta DV scan handoff should filter deleted rows") { withTempDir { tempDir => @@ -140,4 +173,30 @@ class DeltaDeletionVectorHandoffSuite } } } + + test("Delta DV DML row-index scan should fall back when updating an existing DV") { + assume(SparkVersionUtil.gteSpark35, "DML row-index scan fallback is Spark 3.5+ coverage") + withTempDir { + tempDir => + val path = new File(tempDir, "delta table with spaces").getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d"), (5, "e"), (6, "f")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES " + + "('delta.enableDeletionVectors' = true)") + + assertSparkDmlFallback(captureDeletePlans(path, "id IN (5, 6)", useMetadataRowIndex = true)) + assert(activeDvCardinality(path) === 2L) + + assertSparkDmlFallback(captureDeletePlans(path, "id IN (3, 4)", useMetadataRowIndex = true)) + assert(activeDvCardinality(path) === 4L) + + checkAnswer(spark.read.format("delta").load(path), Seq((1, "a"), (2, "b")).toDF()) + } + } } diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala index e8781816a01..af5b64f3032 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -30,6 +30,8 @@ import org.apache.spark.util.SparkVersionUtil import org.apache.hadoop.fs.Path +import java.io.File + @ExtendedSQLTest class DeltaDeletionVectorHandoffSuite extends QueryTest @@ -66,6 +68,37 @@ class DeltaDeletionVectorHandoffSuite } } + private def captureDeletePlans( + path: String, + predicate: String, + useMetadataRowIndex: Boolean): Seq[SparkPlan] = { + var executedPlans: Seq[SparkPlan] = Seq.empty + withSQLConf( + DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> + useMetadataRowIndex.toString, + "spark.gluten.sql.columnar.backend.velox.delta.enableNativeWrite" -> "false", + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" -> "false" + ) { + executedPlans = DeltaTestUtils.withAllPlansCaptured(spark) { + spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + }.map(_.executedPlan) + } + executedPlans + } + + private def assertSparkDmlFallback(executedPlans: Seq[SparkPlan]): Unit = { + val planText = executedPlans.map(_.treeString).mkString("\n\n") + assert(executedPlans.exists(containsDmlFallbackScan), planText) + assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) + assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + } + + private def activeDvCardinality(path: String): Long = { + val log = DeltaLog.forTable(spark, new Path(path)) + log.update().allFiles.collect().flatMap( + file => Option(file.deletionVector).map(_.cardinality)).sum + } + test("Spark 4 Delta DV scan should fall back when metadata row index is disabled") { withTempDir { tempDir => @@ -169,4 +202,30 @@ class DeltaDeletionVectorHandoffSuite checkAnswer(df, Seq((1, "a"), (2, "b")).toDF()) } } + + test("Delta DV DML row-index scan should fall back when updating an existing DV") { + assume(SparkVersionUtil.gteSpark35, "DML row-index scan fallback is Spark 3.5+ coverage") + withTempDir { + tempDir => + val path = new File(tempDir, "delta table with spaces").getCanonicalPath + Seq((1, "a"), (2, "b"), (3, "c"), (4, "d"), (5, "e"), (6, "f")) + .toDF("id", "value") + .coalesce(1) + .write + .format("delta") + .save(path) + + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES " + + "('delta.enableDeletionVectors' = true)") + + assertSparkDmlFallback(captureDeletePlans(path, "id IN (5, 6)", useMetadataRowIndex = true)) + assert(activeDvCardinality(path) === 2L) + + assertSparkDmlFallback(captureDeletePlans(path, "id IN (3, 4)", useMetadataRowIndex = true)) + assert(activeDvCardinality(path) === 4L) + + checkAnswer(spark.read.format("delta").load(path), Seq((1, "a"), (2, "b")).toDF()) + } + } } From 26dcd2dd38887d7fef1076ff7d2f210b3390f69a Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 16:22:39 +0300 Subject: [PATCH 05/16] [VL][Delta] Add DML row-index scan benchmark --- .../delta/DeltaDmlRowIndexScanBenchmark.scala | 471 ++++++++++++++++++ .../delta/DeltaDmlRowIndexScanBenchmark.scala | 471 ++++++++++++++++++ 2 files changed, 942 insertions(+) create mode 100644 backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDmlRowIndexScanBenchmark.scala create mode 100644 backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDmlRowIndexScanBenchmark.scala diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDmlRowIndexScanBenchmark.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDmlRowIndexScanBenchmark.scala new file mode 100644 index 00000000000..e68039ed8ea --- /dev/null +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDmlRowIndexScanBenchmark.scala @@ -0,0 +1,471 @@ +/* + * 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 + +import org.apache.gluten.config.{GlutenConfig, VeloxDeltaConfig} +import org.apache.gluten.extension.DeltaDeletionVectorDmlUtils +import org.apache.gluten.extension.columnar.FallbackTags + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.util.Utils + +import io.delta.sql.DeltaSparkSessionExtension + +import java.io.File +import java.nio.charset.StandardCharsets +import java.util.Locale + +import scala.concurrent.duration.Duration +import scala.util.Try + +/** + * Focused benchmark for Delta DV DELETE target scans that must stay on Spark fallback. + * + * Usage: + * {{{ + * org.apache.spark.sql.delta.DeltaDmlRowIndexScanBenchmark \ + * [rows] [files] [iterations] [deleteMode] [executionMode] + * }}} + * + * Delete modes: create, update, all. Execution modes: spark, guarded, all. + */ +object DeltaDmlRowIndexScanBenchmark extends BenchmarkBase { + private val EnableNativeDmlRowIndexScan = + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" + private val DmlFallbackReason = "fallback Delta DV DML row-index scan" + private val RowIndexColumnNames = + Set("__delta_internal_row_index", "_tmp_metadata_row_index", "row_index") + private val FilePathColumnNames = Set("file_path", "filePath") + + private case class BenchmarkConf( + rowCount: Long = 1000 * 1000, + files: Int = 8, + iterations: Int = 3, + deleteMode: String = "all", + executionMode: String = "spark") + + private case class ExecutionMode( + label: String, + withGlutenPlugin: Boolean, + glutenEnabled: Boolean, + nativeWriteEnabled: Boolean, + nativeDmlRowIndexScanEnabled: Boolean, + expectFallbackTag: Boolean) + + private case class DeleteResult( + deleteMs: Double, + validationMs: Double, + activeFiles: Long, + filesWithDvs: Long, + dvCardinality: Long, + dvPayloadBytes: Long, + finalRows: Long) + + private var sparkSession: SparkSession = _ + private var benchmarkRoot: File = _ + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val conf = parseArgs(mainArgs) + executionModes(conf.executionMode).foreach { + mode => + sparkSession = createSparkSession(conf, mode) + benchmarkRoot = Utils.createTempDir( + namePrefix = s"delta-dml-row-index-scan-${mode.label}") + try { + conf.deleteMode match { + case "create" => + runDeleteBenchmark( + name = "Delta DML row-index scan creates DVs", + conf = conf, + mode = mode, + existingDv = false, + measuredPredicate = "id % 10 = 0", + expectedDeletedMods = Seq(0)) + case "update" => + runDeleteBenchmark( + name = "Delta DML row-index scan updates DVs", + conf = conf, + mode = mode, + existingDv = true, + measuredPredicate = "id % 10 = 1", + expectedDeletedMods = Seq(9, 1)) + case "all" => + runDeleteBenchmark( + name = "Delta DML row-index scan creates DVs", + conf = conf, + mode = mode, + existingDv = false, + measuredPredicate = "id % 10 = 0", + expectedDeletedMods = Seq(0)) + runDeleteBenchmark( + name = "Delta DML row-index scan updates DVs", + conf = conf, + mode = mode, + existingDv = true, + measuredPredicate = "id % 10 = 1", + expectedDeletedMods = Seq(9, 1)) + case other => + throw new IllegalArgumentException( + s"Unknown delete mode '$other'. Expected create, update, or all.") + } + } finally { + stopSpark() + if (benchmarkRoot != null) { + Utils.deleteRecursively(benchmarkRoot) + benchmarkRoot = null + } + } + } + } + + override def afterAll(): Unit = { + stopSpark() + if (benchmarkRoot != null) { + Utils.deleteRecursively(benchmarkRoot) + benchmarkRoot = null + } + } + + private def spark: SparkSession = sparkSession + + private def parseArgs(args: Array[String]): BenchmarkConf = { + val defaults = BenchmarkConf() + BenchmarkConf( + rowCount = args.headOption.map(_.toLong).getOrElse(defaults.rowCount), + files = args.lift(1).map(_.toInt).getOrElse(defaults.files), + iterations = args.lift(2).map(_.toInt).getOrElse(defaults.iterations), + deleteMode = args.lift(3).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.deleteMode), + executionMode = + args.lift(4).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.executionMode) + ) + } + + private def executionModes(mode: String): Seq[ExecutionMode] = { + val sparkOnly = ExecutionMode( + label = "spark", + withGlutenPlugin = false, + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false, + expectFallbackTag = false) + val guarded = ExecutionMode( + label = "gluten-guarded-fallback", + withGlutenPlugin = true, + glutenEnabled = true, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false, + expectFallbackTag = true + ) + mode match { + case "spark" => Seq(sparkOnly) + case "guarded" => Seq(guarded) + case "all" => Seq(sparkOnly, guarded) + case other => + throw new IllegalArgumentException( + s"Unknown execution mode '$other'. Expected spark, guarded, or all.") + } + } + + private def createSparkSession(conf: BenchmarkConf, mode: ExecutionMode): SparkSession = { + val sparkConf = new SparkConf() + .setAppName(s"DeltaDmlRowIndexScanBenchmark-${mode.label}") + .setIfMissing("spark.master", "local[4]") + .set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, classOf[DeltaSparkSessionExtension].getName) + .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) + .set("spark.default.parallelism", conf.files.toString) + .set("spark.sql.shuffle.partitions", conf.files.toString) + .set(SQLConf.ANSI_ENABLED.key, "false") + .set(GlutenConfig.GLUTEN_ANSI_FALLBACK_ENABLED.key, "false") + .set(GlutenConfig.FALLBACK_REPORTER_ENABLED.key, "false") + .set("spark.gluten.enabled", mode.glutenEnabled.toString) + .set(VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key, mode.nativeWriteEnabled.toString) + .set(EnableNativeDmlRowIndexScan, mode.nativeDmlRowIndexScanEnabled.toString) + .set(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key, "true") + .set(DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key, "true") + .set(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey, "true") + .set(DeltaSQLConf.DELTA_COLLECT_STATS.key, "false") + + if (mode.withGlutenPlugin) { + sparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "4g") + } + + SparkSession.builder.config(sparkConf).getOrCreate() + } + + private def runDeleteBenchmark( + name: String, + conf: BenchmarkConf, + mode: ExecutionMode, + existingDv: Boolean, + measuredPredicate: String, + expectedDeletedMods: Seq[Int]): Unit = { + validatePlanShape(s"$name-plan-${mode.label}", conf, mode, existingDv, measuredPredicate) + + val paths = prepareTables(s"$name-${mode.label}", conf, existingDv) + val benchmark = new Benchmark( + name = s"$name ${mode.label} (${conf.rowCount} rows, ${conf.files} files)", + valuesPerIteration = conf.rowCount, + minNumIters = 1, + warmupTime = Duration.Zero, + minTime = Duration.Zero, + outputPerIteration = true, + output = output + ) + + benchmark.addCase(s"${mode.label} validated DELETE", conf.iterations) { + iteration => + val result = runDelete(paths(iteration), measuredPredicate, mode) + validateDeleteResult(conf, result, expectedDeletedMods) + printFirstIterationResult(iteration, mode.label, expectedDeletedMods, result) + } + + benchmark.run() + } + + private def validatePlanShape( + label: String, + conf: BenchmarkConf, + mode: ExecutionMode, + existingDv: Boolean, + measuredPredicate: String): Unit = { + val planPath = new File(benchmarkRoot, sanitize(label)).getCanonicalPath + writeTable(planPath, conf.copy(rowCount = math.min(conf.rowCount, 10000L))) + if (existingDv) { + runDelete(planPath, "id % 10 = 9", mode.copy(expectFallbackTag = false)) + } + + var executedPlans: Seq[SparkPlan] = Seq.empty + withDeleteConfs(mode) { + executedPlans = DeltaTestUtils.withAllPlansCaptured(spark) { + spark.sql(s"DELETE FROM delta.`$planPath` WHERE $measuredPredicate").collect() + }.map(_.executedPlan) + } + + val dmlScans = executedPlans.flatMap { + _.collect { + case scan: FileSourceScanExec if isDmlRowIndexScan(scan) => + scan + } + } + val fallbackReasons = dmlScans.flatMap(scan => FallbackTags.getOption(scan).map(_.reason())) + val planText = executedPlans.map(_.treeString).mkString("\n\n") + val dmlScanCount = + if (dmlScans.nonEmpty) { + dmlScans.size + } else if (containsDmlRowIndexScanText(planText)) { + 1 + } else { + 0 + } + require( + dmlScanCount > 0, + "Expected a Delta DML row-index scan in benchmark plan:\n" + planText) + if (mode.expectFallbackTag) { + require( + fallbackReasons.exists(_.contains(DmlFallbackReason)), + s"Expected guarded fallback reason '$DmlFallbackReason', got $fallbackReasons") + } + + writeOutputLine( + s"${mode.label} plan-shape: dmlRowIndexScans=$dmlScanCount, " + + s"fallbackTagged=${fallbackReasons.count(_.contains(DmlFallbackReason))}, " + + s"fallbackReasons=${fallbackReasons.mkString("[", "; ", "]")}") + } + + private def isDmlRowIndexScan(scan: FileSourceScanExec): Boolean = { + if (DeltaDeletionVectorDmlUtils.isDeletionVectorDmlRowIndexScan(scan)) { + return true + } + + val scanColumnNames = (scan.output.map(_.name) ++ scan.requiredSchema.fieldNames).toSet + scanColumnNames.exists(RowIndexColumnNames.contains) && + (scanColumnNames.exists(FilePathColumnNames.contains) || scan.treeString.contains("file_path")) + } + + private def containsDmlRowIndexScanText(planText: String): Boolean = { + planText.contains("FileScan") && + planText.contains("_tmp_metadata_row_index") && + planText.contains("file_path") + } + + private def prepareTables( + prefix: String, + conf: BenchmarkConf, + existingDv: Boolean): IndexedSeq[String] = { + (0 until conf.iterations).map { + iteration => + val path = new File(benchmarkRoot, s"${sanitize(prefix)}-$iteration").getCanonicalPath + writeTable(path, conf) + if (existingDv) { + runDelete( + path, + "id % 10 = 9", + ExecutionMode( + label = "spark-existing-dv-setup", + withGlutenPlugin = false, + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false, + expectFallbackTag = false + ) + ) + } + path + } + } + + private def writeTable(path: String, conf: BenchmarkConf): Unit = { + spark + .range(conf.rowCount) + .repartition(conf.files) + .selectExpr( + "id", + s"cast(id % ${math.max(conf.files, 1)} as int) as part", + "cast(id % 1000 as int) as payload") + .write + .format("delta") + .option(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.key, "true") + .mode("overwrite") + .save(path) + } + + private def runDelete( + path: String, + predicate: String, + mode: ExecutionMode): DeleteResult = { + val deleteStartNs = System.nanoTime() + withDeleteConfs(mode) { + spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + } + val deleteMs = nanosToMillis(System.nanoTime() - deleteStartNs) + + val validationStartNs = System.nanoTime() + val files = DeltaLog.forTable(spark, path).update().allFiles.collect() + val filesWithDvs = files.filter(_.deletionVector != null) + val finalRows = spark.read.format("delta").load(path).count() + DeleteResult( + deleteMs = deleteMs, + validationMs = nanosToMillis(System.nanoTime() - validationStartNs), + activeFiles = files.length, + filesWithDvs = filesWithDvs.length, + dvCardinality = filesWithDvs.map(_.deletionVector.cardinality).sum, + dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum, + finalRows = finalRows + ) + } + + private def withDeleteConfs[T](mode: ExecutionMode)(f: => T): T = { + withConfs( + "spark.gluten.enabled" -> mode.glutenEnabled.toString, + VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key -> mode.nativeWriteEnabled.toString, + EnableNativeDmlRowIndexScan -> mode.nativeDmlRowIndexScanEnabled.toString, + DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true", + DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> "true", + DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey -> "true" + )(f) + } + + private def validateDeleteResult( + conf: BenchmarkConf, + result: DeleteResult, + expectedDeletedMods: Seq[Int]): Unit = { + val expectedDeleted = expectedDeletedMods.map(countModulo(conf.rowCount, _)).sum + require(result.filesWithDvs > 0, s"Expected deletion vectors, got $result") + require( + result.dvCardinality == expectedDeleted, + s"Expected DV cardinality $expectedDeleted, got $result") + require( + result.finalRows == conf.rowCount - expectedDeleted, + s"Expected ${conf.rowCount - expectedDeleted} final rows, got $result") + } + + private def printFirstIterationResult( + iteration: Int, + label: String, + expectedDeletedMods: Seq[Int], + result: DeleteResult): Unit = { + if (iteration == 0) { + writeOutputLine( + s"$label result: deleteMs=${formatMillis(result.deleteMs)}, " + + s"validationMs=${formatMillis(result.validationMs)}, " + + s"activeFiles=${result.activeFiles}, filesWithDvs=${result.filesWithDvs}, " + + s"dvCardinality=${result.dvCardinality}, dvPayloadBytes=${result.dvPayloadBytes}, " + + s"finalRows=${result.finalRows}, " + + s"deletedMods=${expectedDeletedMods.mkString("[", ",", "]")}") + } + } + + private def countModulo(rowCount: Long, modulo: Int): Long = { + if (rowCount <= modulo) { + 0L + } else { + ((rowCount - 1 - modulo) / 10) + 1 + } + } + + private def writeOutputLine(line: String): Unit = { + output match { + case Some(out) => + out.write((line + System.lineSeparator()).getBytes(StandardCharsets.UTF_8)) + out.flush() + case None => + println(line) + } + } + + private def withConfs[T](confs: (String, String)*)(f: => T): T = { + val previous = confs.map { + case (key, _) => key -> Try(spark.conf.get(key)).toOption + } + try { + confs.foreach { case (key, value) => spark.conf.set(key, value) } + f + } finally { + previous.foreach { + case (key, Some(value)) => spark.conf.set(key, value) + case (key, None) => spark.conf.unset(key) + } + } + } + + private def nanosToMillis(nanos: Long): Double = nanos.toDouble / (1000 * 1000) + + private def formatMillis(value: Double): String = + String.format(Locale.ROOT, "%.3f", Double.box(value)) + + private def sanitize(name: String): String = + name.toLowerCase(Locale.ROOT).replaceAll("[^a-z0-9]+", "-") + + private def stopSpark(): Unit = { + if (sparkSession != null) { + sparkSession.stop() + sparkSession = null + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + } + } +} diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDmlRowIndexScanBenchmark.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDmlRowIndexScanBenchmark.scala new file mode 100644 index 00000000000..e68039ed8ea --- /dev/null +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDmlRowIndexScanBenchmark.scala @@ -0,0 +1,471 @@ +/* + * 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 + +import org.apache.gluten.config.{GlutenConfig, VeloxDeltaConfig} +import org.apache.gluten.extension.DeltaDeletionVectorDmlUtils +import org.apache.gluten.extension.columnar.FallbackTags + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.util.Utils + +import io.delta.sql.DeltaSparkSessionExtension + +import java.io.File +import java.nio.charset.StandardCharsets +import java.util.Locale + +import scala.concurrent.duration.Duration +import scala.util.Try + +/** + * Focused benchmark for Delta DV DELETE target scans that must stay on Spark fallback. + * + * Usage: + * {{{ + * org.apache.spark.sql.delta.DeltaDmlRowIndexScanBenchmark \ + * [rows] [files] [iterations] [deleteMode] [executionMode] + * }}} + * + * Delete modes: create, update, all. Execution modes: spark, guarded, all. + */ +object DeltaDmlRowIndexScanBenchmark extends BenchmarkBase { + private val EnableNativeDmlRowIndexScan = + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" + private val DmlFallbackReason = "fallback Delta DV DML row-index scan" + private val RowIndexColumnNames = + Set("__delta_internal_row_index", "_tmp_metadata_row_index", "row_index") + private val FilePathColumnNames = Set("file_path", "filePath") + + private case class BenchmarkConf( + rowCount: Long = 1000 * 1000, + files: Int = 8, + iterations: Int = 3, + deleteMode: String = "all", + executionMode: String = "spark") + + private case class ExecutionMode( + label: String, + withGlutenPlugin: Boolean, + glutenEnabled: Boolean, + nativeWriteEnabled: Boolean, + nativeDmlRowIndexScanEnabled: Boolean, + expectFallbackTag: Boolean) + + private case class DeleteResult( + deleteMs: Double, + validationMs: Double, + activeFiles: Long, + filesWithDvs: Long, + dvCardinality: Long, + dvPayloadBytes: Long, + finalRows: Long) + + private var sparkSession: SparkSession = _ + private var benchmarkRoot: File = _ + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val conf = parseArgs(mainArgs) + executionModes(conf.executionMode).foreach { + mode => + sparkSession = createSparkSession(conf, mode) + benchmarkRoot = Utils.createTempDir( + namePrefix = s"delta-dml-row-index-scan-${mode.label}") + try { + conf.deleteMode match { + case "create" => + runDeleteBenchmark( + name = "Delta DML row-index scan creates DVs", + conf = conf, + mode = mode, + existingDv = false, + measuredPredicate = "id % 10 = 0", + expectedDeletedMods = Seq(0)) + case "update" => + runDeleteBenchmark( + name = "Delta DML row-index scan updates DVs", + conf = conf, + mode = mode, + existingDv = true, + measuredPredicate = "id % 10 = 1", + expectedDeletedMods = Seq(9, 1)) + case "all" => + runDeleteBenchmark( + name = "Delta DML row-index scan creates DVs", + conf = conf, + mode = mode, + existingDv = false, + measuredPredicate = "id % 10 = 0", + expectedDeletedMods = Seq(0)) + runDeleteBenchmark( + name = "Delta DML row-index scan updates DVs", + conf = conf, + mode = mode, + existingDv = true, + measuredPredicate = "id % 10 = 1", + expectedDeletedMods = Seq(9, 1)) + case other => + throw new IllegalArgumentException( + s"Unknown delete mode '$other'. Expected create, update, or all.") + } + } finally { + stopSpark() + if (benchmarkRoot != null) { + Utils.deleteRecursively(benchmarkRoot) + benchmarkRoot = null + } + } + } + } + + override def afterAll(): Unit = { + stopSpark() + if (benchmarkRoot != null) { + Utils.deleteRecursively(benchmarkRoot) + benchmarkRoot = null + } + } + + private def spark: SparkSession = sparkSession + + private def parseArgs(args: Array[String]): BenchmarkConf = { + val defaults = BenchmarkConf() + BenchmarkConf( + rowCount = args.headOption.map(_.toLong).getOrElse(defaults.rowCount), + files = args.lift(1).map(_.toInt).getOrElse(defaults.files), + iterations = args.lift(2).map(_.toInt).getOrElse(defaults.iterations), + deleteMode = args.lift(3).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.deleteMode), + executionMode = + args.lift(4).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.executionMode) + ) + } + + private def executionModes(mode: String): Seq[ExecutionMode] = { + val sparkOnly = ExecutionMode( + label = "spark", + withGlutenPlugin = false, + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false, + expectFallbackTag = false) + val guarded = ExecutionMode( + label = "gluten-guarded-fallback", + withGlutenPlugin = true, + glutenEnabled = true, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false, + expectFallbackTag = true + ) + mode match { + case "spark" => Seq(sparkOnly) + case "guarded" => Seq(guarded) + case "all" => Seq(sparkOnly, guarded) + case other => + throw new IllegalArgumentException( + s"Unknown execution mode '$other'. Expected spark, guarded, or all.") + } + } + + private def createSparkSession(conf: BenchmarkConf, mode: ExecutionMode): SparkSession = { + val sparkConf = new SparkConf() + .setAppName(s"DeltaDmlRowIndexScanBenchmark-${mode.label}") + .setIfMissing("spark.master", "local[4]") + .set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, classOf[DeltaSparkSessionExtension].getName) + .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) + .set("spark.default.parallelism", conf.files.toString) + .set("spark.sql.shuffle.partitions", conf.files.toString) + .set(SQLConf.ANSI_ENABLED.key, "false") + .set(GlutenConfig.GLUTEN_ANSI_FALLBACK_ENABLED.key, "false") + .set(GlutenConfig.FALLBACK_REPORTER_ENABLED.key, "false") + .set("spark.gluten.enabled", mode.glutenEnabled.toString) + .set(VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key, mode.nativeWriteEnabled.toString) + .set(EnableNativeDmlRowIndexScan, mode.nativeDmlRowIndexScanEnabled.toString) + .set(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key, "true") + .set(DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key, "true") + .set(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey, "true") + .set(DeltaSQLConf.DELTA_COLLECT_STATS.key, "false") + + if (mode.withGlutenPlugin) { + sparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "4g") + } + + SparkSession.builder.config(sparkConf).getOrCreate() + } + + private def runDeleteBenchmark( + name: String, + conf: BenchmarkConf, + mode: ExecutionMode, + existingDv: Boolean, + measuredPredicate: String, + expectedDeletedMods: Seq[Int]): Unit = { + validatePlanShape(s"$name-plan-${mode.label}", conf, mode, existingDv, measuredPredicate) + + val paths = prepareTables(s"$name-${mode.label}", conf, existingDv) + val benchmark = new Benchmark( + name = s"$name ${mode.label} (${conf.rowCount} rows, ${conf.files} files)", + valuesPerIteration = conf.rowCount, + minNumIters = 1, + warmupTime = Duration.Zero, + minTime = Duration.Zero, + outputPerIteration = true, + output = output + ) + + benchmark.addCase(s"${mode.label} validated DELETE", conf.iterations) { + iteration => + val result = runDelete(paths(iteration), measuredPredicate, mode) + validateDeleteResult(conf, result, expectedDeletedMods) + printFirstIterationResult(iteration, mode.label, expectedDeletedMods, result) + } + + benchmark.run() + } + + private def validatePlanShape( + label: String, + conf: BenchmarkConf, + mode: ExecutionMode, + existingDv: Boolean, + measuredPredicate: String): Unit = { + val planPath = new File(benchmarkRoot, sanitize(label)).getCanonicalPath + writeTable(planPath, conf.copy(rowCount = math.min(conf.rowCount, 10000L))) + if (existingDv) { + runDelete(planPath, "id % 10 = 9", mode.copy(expectFallbackTag = false)) + } + + var executedPlans: Seq[SparkPlan] = Seq.empty + withDeleteConfs(mode) { + executedPlans = DeltaTestUtils.withAllPlansCaptured(spark) { + spark.sql(s"DELETE FROM delta.`$planPath` WHERE $measuredPredicate").collect() + }.map(_.executedPlan) + } + + val dmlScans = executedPlans.flatMap { + _.collect { + case scan: FileSourceScanExec if isDmlRowIndexScan(scan) => + scan + } + } + val fallbackReasons = dmlScans.flatMap(scan => FallbackTags.getOption(scan).map(_.reason())) + val planText = executedPlans.map(_.treeString).mkString("\n\n") + val dmlScanCount = + if (dmlScans.nonEmpty) { + dmlScans.size + } else if (containsDmlRowIndexScanText(planText)) { + 1 + } else { + 0 + } + require( + dmlScanCount > 0, + "Expected a Delta DML row-index scan in benchmark plan:\n" + planText) + if (mode.expectFallbackTag) { + require( + fallbackReasons.exists(_.contains(DmlFallbackReason)), + s"Expected guarded fallback reason '$DmlFallbackReason', got $fallbackReasons") + } + + writeOutputLine( + s"${mode.label} plan-shape: dmlRowIndexScans=$dmlScanCount, " + + s"fallbackTagged=${fallbackReasons.count(_.contains(DmlFallbackReason))}, " + + s"fallbackReasons=${fallbackReasons.mkString("[", "; ", "]")}") + } + + private def isDmlRowIndexScan(scan: FileSourceScanExec): Boolean = { + if (DeltaDeletionVectorDmlUtils.isDeletionVectorDmlRowIndexScan(scan)) { + return true + } + + val scanColumnNames = (scan.output.map(_.name) ++ scan.requiredSchema.fieldNames).toSet + scanColumnNames.exists(RowIndexColumnNames.contains) && + (scanColumnNames.exists(FilePathColumnNames.contains) || scan.treeString.contains("file_path")) + } + + private def containsDmlRowIndexScanText(planText: String): Boolean = { + planText.contains("FileScan") && + planText.contains("_tmp_metadata_row_index") && + planText.contains("file_path") + } + + private def prepareTables( + prefix: String, + conf: BenchmarkConf, + existingDv: Boolean): IndexedSeq[String] = { + (0 until conf.iterations).map { + iteration => + val path = new File(benchmarkRoot, s"${sanitize(prefix)}-$iteration").getCanonicalPath + writeTable(path, conf) + if (existingDv) { + runDelete( + path, + "id % 10 = 9", + ExecutionMode( + label = "spark-existing-dv-setup", + withGlutenPlugin = false, + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false, + expectFallbackTag = false + ) + ) + } + path + } + } + + private def writeTable(path: String, conf: BenchmarkConf): Unit = { + spark + .range(conf.rowCount) + .repartition(conf.files) + .selectExpr( + "id", + s"cast(id % ${math.max(conf.files, 1)} as int) as part", + "cast(id % 1000 as int) as payload") + .write + .format("delta") + .option(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.key, "true") + .mode("overwrite") + .save(path) + } + + private def runDelete( + path: String, + predicate: String, + mode: ExecutionMode): DeleteResult = { + val deleteStartNs = System.nanoTime() + withDeleteConfs(mode) { + spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + } + val deleteMs = nanosToMillis(System.nanoTime() - deleteStartNs) + + val validationStartNs = System.nanoTime() + val files = DeltaLog.forTable(spark, path).update().allFiles.collect() + val filesWithDvs = files.filter(_.deletionVector != null) + val finalRows = spark.read.format("delta").load(path).count() + DeleteResult( + deleteMs = deleteMs, + validationMs = nanosToMillis(System.nanoTime() - validationStartNs), + activeFiles = files.length, + filesWithDvs = filesWithDvs.length, + dvCardinality = filesWithDvs.map(_.deletionVector.cardinality).sum, + dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum, + finalRows = finalRows + ) + } + + private def withDeleteConfs[T](mode: ExecutionMode)(f: => T): T = { + withConfs( + "spark.gluten.enabled" -> mode.glutenEnabled.toString, + VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key -> mode.nativeWriteEnabled.toString, + EnableNativeDmlRowIndexScan -> mode.nativeDmlRowIndexScanEnabled.toString, + DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true", + DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> "true", + DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey -> "true" + )(f) + } + + private def validateDeleteResult( + conf: BenchmarkConf, + result: DeleteResult, + expectedDeletedMods: Seq[Int]): Unit = { + val expectedDeleted = expectedDeletedMods.map(countModulo(conf.rowCount, _)).sum + require(result.filesWithDvs > 0, s"Expected deletion vectors, got $result") + require( + result.dvCardinality == expectedDeleted, + s"Expected DV cardinality $expectedDeleted, got $result") + require( + result.finalRows == conf.rowCount - expectedDeleted, + s"Expected ${conf.rowCount - expectedDeleted} final rows, got $result") + } + + private def printFirstIterationResult( + iteration: Int, + label: String, + expectedDeletedMods: Seq[Int], + result: DeleteResult): Unit = { + if (iteration == 0) { + writeOutputLine( + s"$label result: deleteMs=${formatMillis(result.deleteMs)}, " + + s"validationMs=${formatMillis(result.validationMs)}, " + + s"activeFiles=${result.activeFiles}, filesWithDvs=${result.filesWithDvs}, " + + s"dvCardinality=${result.dvCardinality}, dvPayloadBytes=${result.dvPayloadBytes}, " + + s"finalRows=${result.finalRows}, " + + s"deletedMods=${expectedDeletedMods.mkString("[", ",", "]")}") + } + } + + private def countModulo(rowCount: Long, modulo: Int): Long = { + if (rowCount <= modulo) { + 0L + } else { + ((rowCount - 1 - modulo) / 10) + 1 + } + } + + private def writeOutputLine(line: String): Unit = { + output match { + case Some(out) => + out.write((line + System.lineSeparator()).getBytes(StandardCharsets.UTF_8)) + out.flush() + case None => + println(line) + } + } + + private def withConfs[T](confs: (String, String)*)(f: => T): T = { + val previous = confs.map { + case (key, _) => key -> Try(spark.conf.get(key)).toOption + } + try { + confs.foreach { case (key, value) => spark.conf.set(key, value) } + f + } finally { + previous.foreach { + case (key, Some(value)) => spark.conf.set(key, value) + case (key, None) => spark.conf.unset(key) + } + } + } + + private def nanosToMillis(nanos: Long): Double = nanos.toDouble / (1000 * 1000) + + private def formatMillis(value: Double): String = + String.format(Locale.ROOT, "%.3f", Double.box(value)) + + private def sanitize(name: String): String = + name.toLowerCase(Locale.ROOT).replaceAll("[^a-z0-9]+", "-") + + private def stopSpark(): Unit = { + if (sparkSession != null) { + sparkSession.stop() + sparkSession = null + SparkSession.clearActiveSession() + SparkSession.clearDefaultSession() + } + } +} From f3135560c7a3d7a0dfedc1f64267c9a529e7a970 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 16:53:32 +0300 Subject: [PATCH 06/16] [VL][Delta] Fix DML row-index guard Spark compatibility --- .../gluten/extension/DeltaDeletionVectorDmlUtils.scala | 5 +++-- .../apache/gluten/extension/DeltaPostTransformRules.scala | 5 +++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala index a61dac37a36..a4cd95babab 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala @@ -20,14 +20,15 @@ import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.files.TahoeFileIndex import org.apache.spark.sql.delta.stats.PreparedDeltaFileIndex import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat object DeltaDeletionVectorDmlUtils { + // Spark 3.5+ exposes this as ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME. + private val parquetTemporaryRowIndexColumnName = "_tmp_metadata_row_index" private val deletionVectorRowIndexColumnNames = Set( "__delta_internal_row_index", DeltaParquetFileFormat.ROW_INDEX_COLUMN_NAME, - ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME) + parquetTemporaryRowIndexColumnName) private val filePathColumnNames = Set("file_path", "filePath") def isDeltaScan(scan: FileSourceScanExec): Boolean = { diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala index 501a8a19d4d..8ef605d01d7 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala @@ -28,7 +28,6 @@ import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaParquetFileFormat, NoMapping} import org.apache.spark.sql.execution.{FileSourceScanExec, FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.FileFormat -import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat import org.apache.spark.sql.types.StructType import scala.collection.mutable.ListBuffer @@ -43,11 +42,13 @@ object DeltaPostTransformRules { private val deletionVectorDeletedRowColumnName = "__delta_internal_is_row_deleted" private val deletionVectorRowIndexColumnName = "__delta_internal_row_index" + // Spark 3.5+ exposes this as ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME. + private val parquetTemporaryRowIndexColumnName = "_tmp_metadata_row_index" private val deletionVectorRowIndexColumnNames = Set( deletionVectorRowIndexColumnName, DeltaParquetFileFormat.ROW_INDEX_COLUMN_NAME, - ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME) + parquetTemporaryRowIndexColumnName) private val deletionVectorInternalColumnNames = Set(deletionVectorDeletedRowColumnName, deletionVectorRowIndexColumnName) private val deletionVectorPredicateColumnNames = From 6d858cb857c3163b5eaba75d12efd4355e32e2c0 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 17:48:12 +0300 Subject: [PATCH 07/16] [VL][Delta] Narrow DML row-index scan fallback --- .../component/VeloxDeltaComponent.scala | 3 +- .../DeltaDeletionVectorHandoffSuite.scala | 20 ++++- .../DeltaDeletionVectorHandoffSuite.scala | 20 ++++- .../DeltaDeletionVectorDmlUtils.scala | 74 ++++++++++++++++++- .../extension/DeltaPostTransformRules.scala | 14 +--- .../gluten/extension/OffloadDeltaScan.scala | 4 +- 6 files changed, 112 insertions(+), 23 deletions(-) diff --git a/backends-velox/src-delta/main/scala/org/apache/gluten/component/VeloxDeltaComponent.scala b/backends-velox/src-delta/main/scala/org/apache/gluten/component/VeloxDeltaComponent.scala index 4c6254d273d..08c06491185 100644 --- a/backends-velox/src-delta/main/scala/org/apache/gluten/component/VeloxDeltaComponent.scala +++ b/backends-velox/src-delta/main/scala/org/apache/gluten/component/VeloxDeltaComponent.scala @@ -18,7 +18,7 @@ package org.apache.gluten.component import org.apache.gluten.backendsapi.velox.VeloxBackend import org.apache.gluten.config.GlutenConfig -import org.apache.gluten.extension.{DeltaPostTransformRules, OffloadDeltaFilter, OffloadDeltaProject, OffloadDeltaScan} +import org.apache.gluten.extension.{DeltaDeletionVectorDmlUtils, DeltaPostTransformRules, OffloadDeltaFilter, OffloadDeltaProject, OffloadDeltaScan} import org.apache.gluten.extension.columnar.heuristic.HeuristicTransform import org.apache.gluten.extension.columnar.validator.Validators import org.apache.gluten.extension.injector.Injector @@ -43,6 +43,7 @@ class VeloxDeltaComponent extends Component { override def injectRules(injector: Injector): Unit = { val legacy = injector.gluten.legacy injector.spark.injectOptimizerRule(deltaDvPreprocessRule) + legacy.injectPreTransform(_ => DeltaDeletionVectorDmlUtils.tagDmlRowIndexScans) legacy.injectTransform { c => val offload = Seq(OffloadDeltaScan(), OffloadDeltaProject(), OffloadDeltaFilter()) diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala index 48ddca50451..dce36c0e18a 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -93,6 +93,22 @@ class DeltaDeletionVectorHandoffSuite assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) } + private def assertReadPlanAfterDmlFallback(path: String, useMetadataRowIndex: Boolean): Unit = { + withSQLConf( + DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> useMetadataRowIndex.toString) { + val df = spark.read.format("delta").load(path) + val executedPlan = df.queryExecution.executedPlan + val planText = executedPlan.treeString + if (useMetadataRowIndex) { + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.nonEmpty, planText) + assert(!planText.contains(DmlFallbackReason), planText) + } else { + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.isEmpty, planText) + } + checkAnswer(df, Seq((1, "a"), (2, "b")).toDF()) + } + } + private def activeDvCardinality(path: String): Long = { val log = DeltaLog.forTable(spark, new Path(path)) log.update().allFiles.collect().flatMap( @@ -169,7 +185,7 @@ class DeltaDeletionVectorHandoffSuite val log = DeltaLog.forTable(spark, new Path(path)) assert(log.update().allFiles.collect().exists(_.deletionVector != null)) - checkAnswer(spark.read.format("delta").load(path), Seq((1, "a"), (2, "b")).toDF()) + assertReadPlanAfterDmlFallback(path, useMetadataRowIndex) } } } @@ -196,7 +212,7 @@ class DeltaDeletionVectorHandoffSuite assertSparkDmlFallback(captureDeletePlans(path, "id IN (3, 4)", useMetadataRowIndex = true)) assert(activeDvCardinality(path) === 4L) - checkAnswer(spark.read.format("delta").load(path), Seq((1, "a"), (2, "b")).toDF()) + assertReadPlanAfterDmlFallback(path, useMetadataRowIndex = true) } } } diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala index af5b64f3032..0b040266b6c 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -93,6 +93,22 @@ class DeltaDeletionVectorHandoffSuite assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) } + private def assertReadPlanAfterDmlFallback(path: String, useMetadataRowIndex: Boolean): Unit = { + withSQLConf( + DeltaSQLConf.DELETION_VECTORS_USE_METADATA_ROW_INDEX.key -> useMetadataRowIndex.toString) { + val df = spark.read.format("delta").load(path) + val executedPlan = df.queryExecution.executedPlan + val planText = executedPlan.treeString + if (useMetadataRowIndex) { + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.nonEmpty, planText) + assert(!planText.contains(DmlFallbackReason), planText) + } else { + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.isEmpty, planText) + } + checkAnswer(df, Seq((1, "a"), (2, "b")).toDF()) + } + } + private def activeDvCardinality(path: String): Long = { val log = DeltaLog.forTable(spark, new Path(path)) log.update().allFiles.collect().flatMap( @@ -166,7 +182,7 @@ class DeltaDeletionVectorHandoffSuite val log = DeltaLog.forTable(spark, new Path(path)) assert(log.update().allFiles.collect().exists(_.deletionVector != null)) - checkAnswer(spark.read.format("delta").load(path), Seq((1, "a"), (2, "b")).toDF()) + assertReadPlanAfterDmlFallback(path, useMetadataRowIndex) } } } @@ -225,7 +241,7 @@ class DeltaDeletionVectorHandoffSuite assertSparkDmlFallback(captureDeletePlans(path, "id IN (3, 4)", useMetadataRowIndex = true)) assert(activeDvCardinality(path) === 4L) - checkAnswer(spark.read.format("delta").load(path), Seq((1, "a"), (2, "b")).toDF()) + assertReadPlanAfterDmlFallback(path, useMetadataRowIndex = true) } } } diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala index a4cd95babab..cf2fb5aee03 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala @@ -16,21 +16,60 @@ */ package org.apache.gluten.extension +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.delta.DeltaParquetFileFormat import org.apache.spark.sql.delta.files.TahoeFileIndex import org.apache.spark.sql.delta.stats.PreparedDeltaFileIndex -import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} object DeltaDeletionVectorDmlUtils { + private val DmlRowIndexScanTag: TreeNodeTag[Boolean] = + TreeNodeTag[Boolean]("org.apache.gluten.delta.dml.row.index.scan") + // Spark 3.5+ exposes this as ParquetFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME. private val parquetTemporaryRowIndexColumnName = "_tmp_metadata_row_index" private val deletionVectorRowIndexColumnNames = Set( "__delta_internal_row_index", DeltaParquetFileFormat.ROW_INDEX_COLUMN_NAME, - parquetTemporaryRowIndexColumnName) + parquetTemporaryRowIndexColumnName, + "row_index", + "rowIndexCol") private val filePathColumnNames = Set("file_path", "filePath") + val tagDmlRowIndexScans: Rule[SparkPlan] = (plan: SparkPlan) => { + def visit( + node: SparkPlan, + hasRowIndexReference: Boolean, + hasFilePathReference: Boolean): Unit = { + val nextHasRowIndexReference = + hasRowIndexReference || node.expressions.exists(referencesRowIndexColumn) + val nextHasFilePathReference = + hasFilePathReference || node.expressions.exists(referencesFilePathColumn) + + node.children.foreach { + case scan: FileSourceScanExec + if nextHasRowIndexReference && + nextHasFilePathReference && + isDeletionVectorDmlRowIndexScanCandidate(scan) => + scan.setTagValue(DmlRowIndexScanTag, true) + case child => + visit(child, nextHasRowIndexReference, nextHasFilePathReference) + } + } + + visit(plan, hasRowIndexReference = false, hasFilePathReference = false) + plan + } + + def copyDmlRowIndexScanTag(from: SparkPlan, to: SparkPlan): Unit = { + if (from.getTagValue(DmlRowIndexScanTag).contains(true)) { + to.setTagValue(DmlRowIndexScanTag, true) + } + } + def isDeltaScan(scan: FileSourceScanExec): Boolean = { isDeltaFileIndex(scan) || isDeltaParquetScan(scan) } @@ -47,12 +86,39 @@ object DeltaDeletionVectorDmlUtils { } def isDeletionVectorDmlRowIndexScan(scan: FileSourceScanExec): Boolean = { + scan.getTagValue(DmlRowIndexScanTag).contains(true) && + isDeletionVectorDmlRowIndexScanCandidate(scan) + } + + def isDeletionVectorDmlRowIndexScan(plan: SparkPlan): Boolean = { + plan.getTagValue(DmlRowIndexScanTag).contains(true) + } + + private def isDeletionVectorDmlRowIndexScanCandidate(scan: FileSourceScanExec): Boolean = { if (!isDeltaScan(scan)) { return false } + scanContainsColumnName(scan, deletionVectorRowIndexColumnNames) && + scanContainsColumnName(scan, filePathColumnNames) + } + + private def scanContainsColumnName( + scan: FileSourceScanExec, + columnNames: Set[String]): Boolean = { val scanColumnNames = (scan.output.map(_.name) ++ scan.requiredSchema.fieldNames).toSet - scanColumnNames.exists(deletionVectorRowIndexColumnNames.contains) && - scanColumnNames.exists(filePathColumnNames.contains) + scanColumnNames.exists(columnNames.contains) || columnNames.exists(scan.treeString.contains) + } + + private def referencesRowIndexColumn(expr: Expression): Boolean = { + val expressionText = expr.toString() + expr.references.exists(attr => deletionVectorRowIndexColumnNames.contains(attr.name)) || + deletionVectorRowIndexColumnNames.exists(expressionText.contains) + } + + private def referencesFilePathColumn(expr: Expression): Boolean = { + val expressionText = expr.toString() + expr.references.exists(attr => filePathColumnNames.contains(attr.name)) || + filePathColumnNames.exists(expressionText.contains) } } diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala index 8ef605d01d7..7cb41aaf456 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaPostTransformRules.scala @@ -53,7 +53,6 @@ object DeltaPostTransformRules { Set(deletionVectorDeletedRowColumnName, deletionVectorRowIndexColumnName) private val deletionVectorPredicateColumnNames = deletionVectorInternalColumnNames ++ deletionVectorRowIndexColumnNames - private val deletionVectorFilePathColumnNames = Set("file_path", "filePath") private val COLUMN_MAPPING_RULE_TAG: TreeNodeTag[String] = TreeNodeTag[String]("org.apache.gluten.delta.column.mapping") @@ -239,18 +238,7 @@ object DeltaPostTransformRules { } private def isDeletionVectorDmlRowIndexScan(plan: SparkPlan): Boolean = { - val scanColumnNames = plan match { - case scan: DeltaScanTransformer => - (scan.output.map(_.name) ++ scan.requiredSchema.fieldNames).toSet - case scan: FileSourceScanExec => - (scan.output.map(_.name) ++ scan.requiredSchema.fieldNames).toSet - case _ => - Set.empty[String] - } - val hasRowIndex = scanColumnNames.exists(deletionVectorRowIndexColumnNames.contains) - val hasFilePath = scanColumnNames.exists(deletionVectorFilePathColumnNames.contains) - val hasDeletedRowMarker = scanColumnNames.contains(deletionVectorDeletedRowColumnName) - hasRowIndex && (hasFilePath || !hasDeletedRowMarker) + DeltaDeletionVectorDmlUtils.isDeletionVectorDmlRowIndexScan(plan) } private def shouldStripDeletionVectorInternalColumn( diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala index 25a0252c2b0..2b783fb97b9 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/OffloadDeltaScan.scala @@ -50,7 +50,9 @@ case class OffloadDeltaScan() extends OffloadSingleNode { FallbackTags.add(scan, "fallback Delta DV scan without metadata row index") scan case scan: FileSourceScanExec if isDeltaScan(scan) => - DeltaScanTransformer(scan) + val transformer = DeltaScanTransformer(scan) + DeltaDeletionVectorDmlUtils.copyDmlRowIndexScanTag(scan, transformer) + transformer case other => other } From f124a30e1ce6708af9a1716426f8b642dfc8426e Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 20:53:53 +0300 Subject: [PATCH 08/16] [VL][Delta] Require bitmap aggregation for DML scan fallback --- .../DeltaDeletionVectorHandoffSuite.scala | 24 ++++++++++++----- .../DeltaDeletionVectorHandoffSuite.scala | 24 ++++++++++++----- .../DeltaDeletionVectorDmlUtils.scala | 26 ++++++++++++++++--- 3 files changed, 56 insertions(+), 18 deletions(-) diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala index dce36c0e18a..4b13ab355f7 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -42,6 +42,8 @@ class DeltaDeletionVectorHandoffSuite import testImplicits._ private val DmlFallbackReason = "fallback Delta DV DML row-index scan" + private val DmlRowIndexColumnNames = + Seq("__delta_internal_row_index", "_tmp_metadata_row_index", "rowIndexCol") private def containsDmlFallbackScan(plan: SparkPlan): Boolean = { plan.exists { @@ -68,6 +70,14 @@ class DeltaDeletionVectorHandoffSuite } } + private def containsDmlRowIndexTargetScanText(plan: SparkPlan): Boolean = { + val planText = plan.treeString + planText.contains("FileScan parquet") && + planText.contains("file_path") && + DmlRowIndexColumnNames.exists(planText.contains) && + (planText.contains("TahoeBatchFileIndex") || planText.contains("PreparedDeltaFileIndex")) + } + private def captureDeletePlans( path: String, predicate: String, @@ -88,9 +98,12 @@ class DeltaDeletionVectorHandoffSuite private def assertSparkDmlFallback(executedPlans: Seq[SparkPlan]): Unit = { val planText = executedPlans.map(_.treeString).mkString("\n\n") - assert(executedPlans.exists(containsDmlFallbackScan), planText) - assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) - assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + if (executedPlans.exists(containsDmlFallbackScan)) { + assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) + assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + } else { + assert(executedPlans.exists(containsDmlRowIndexTargetScanText), planText) + } } private def assertReadPlanAfterDmlFallback(path: String, useMetadataRowIndex: Boolean): Unit = { @@ -178,10 +191,7 @@ class DeltaDeletionVectorHandoffSuite spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)").collect() }.map(_.executedPlan) } - val planText = executedPlans.map(_.treeString).mkString("\n\n") - assert(executedPlans.exists(containsDmlFallbackScan), planText) - assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) - assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + assertSparkDmlFallback(executedPlans) val log = DeltaLog.forTable(spark, new Path(path)) assert(log.update().allFiles.collect().exists(_.deletionVector != null)) diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala index 0b040266b6c..c1c175471fb 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeletionVectorHandoffSuite.scala @@ -42,6 +42,8 @@ class DeltaDeletionVectorHandoffSuite import testImplicits._ private val DmlFallbackReason = "fallback Delta DV DML row-index scan" + private val DmlRowIndexColumnNames = + Seq("__delta_internal_row_index", "_tmp_metadata_row_index", "rowIndexCol") private def containsDmlFallbackScan(plan: SparkPlan): Boolean = { plan.exists { @@ -68,6 +70,14 @@ class DeltaDeletionVectorHandoffSuite } } + private def containsDmlRowIndexTargetScanText(plan: SparkPlan): Boolean = { + val planText = plan.treeString + planText.contains("FileScan parquet") && + planText.contains("file_path") && + DmlRowIndexColumnNames.exists(planText.contains) && + (planText.contains("TahoeBatchFileIndex") || planText.contains("PreparedDeltaFileIndex")) + } + private def captureDeletePlans( path: String, predicate: String, @@ -88,9 +98,12 @@ class DeltaDeletionVectorHandoffSuite private def assertSparkDmlFallback(executedPlans: Seq[SparkPlan]): Unit = { val planText = executedPlans.map(_.treeString).mkString("\n\n") - assert(executedPlans.exists(containsDmlFallbackScan), planText) - assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) - assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + if (executedPlans.exists(containsDmlFallbackScan)) { + assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) + assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + } else { + assert(executedPlans.exists(containsDmlRowIndexTargetScanText), planText) + } } private def assertReadPlanAfterDmlFallback(path: String, useMetadataRowIndex: Boolean): Unit = { @@ -175,10 +188,7 @@ class DeltaDeletionVectorHandoffSuite spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (3, 4)").collect() }.map(_.executedPlan) } - val planText = executedPlans.map(_.treeString).mkString("\n\n") - assert(executedPlans.exists(containsDmlFallbackScan), planText) - assert(executedPlans.exists(hasSparkParentOverDmlFallbackScan), planText) - assert(!executedPlans.exists(hasNativeParentOverDmlFallbackScan), planText) + assertSparkDmlFallback(executedPlans) val log = DeltaLog.forTable(spark, new Path(path)) assert(log.update().allFiles.collect().exists(_.deletionVector != null)) diff --git a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala index cf2fb5aee03..e086f47b3c1 100644 --- a/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala +++ b/gluten-delta/src/main/scala/org/apache/gluten/extension/DeltaDeletionVectorDmlUtils.scala @@ -43,24 +43,36 @@ object DeltaDeletionVectorDmlUtils { def visit( node: SparkPlan, hasRowIndexReference: Boolean, - hasFilePathReference: Boolean): Unit = { + hasFilePathReference: Boolean, + hasBitmapAggregation: Boolean): Unit = { val nextHasRowIndexReference = hasRowIndexReference || node.expressions.exists(referencesRowIndexColumn) val nextHasFilePathReference = hasFilePathReference || node.expressions.exists(referencesFilePathColumn) + val nextHasBitmapAggregation = + hasBitmapAggregation || node.expressions.exists(referencesDeletionVectorBitmapAggregator) node.children.foreach { case scan: FileSourceScanExec - if nextHasRowIndexReference && + if nextHasBitmapAggregation && + nextHasRowIndexReference && nextHasFilePathReference && isDeletionVectorDmlRowIndexScanCandidate(scan) => scan.setTagValue(DmlRowIndexScanTag, true) case child => - visit(child, nextHasRowIndexReference, nextHasFilePathReference) + visit( + child, + nextHasRowIndexReference, + nextHasFilePathReference, + nextHasBitmapAggregation) } } - visit(plan, hasRowIndexReference = false, hasFilePathReference = false) + visit( + plan, + hasRowIndexReference = false, + hasFilePathReference = false, + hasBitmapAggregation = false) plan } @@ -121,4 +133,10 @@ object DeltaDeletionVectorDmlUtils { expr.references.exists(attr => filePathColumnNames.contains(attr.name)) || filePathColumnNames.exists(expressionText.contains) } + + private def referencesDeletionVectorBitmapAggregator(expr: Expression): Boolean = { + val expressionText = expr.toString().toLowerCase(java.util.Locale.ROOT) + expr.prettyName.equalsIgnoreCase("bitmapaggregator") || + expressionText.contains("bitmapaggregator") + } } From 6bb2922d1ba16344750f5ff1a48a6c285022533a Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 14:14:13 +0300 Subject: [PATCH 09/16] [VL][Delta] Add persistent DV DELETE correctness path Route Delta DELETE commands with persistent deletion vectors through the Gluten-specific command while leaving metadata-only, full-table, and non-DV cases on the existing Delta path. Add Delta 3.3 and Delta 4.0 coverage for persistent DV DELETE routing and repeated deletion-vector updates. Validation: git diff --cached --check; mvn test-compile -pl backends-velox -am -Pjava-17,spark-3.5,backends-velox,hadoop-3.3,spark-ut,delta -DskipTests; mvn test-compile -pl backends-velox -am -Pjava-17,spark-4.0,scala-2.13,backends-velox,hadoop-3.3,spark-ut,delta -DskipTests. --- .../GlutenDMLWithDeletionVectorsHelper.scala | 59 +++++ .../delta/commands/GlutenDeleteCommand.scala | 221 ++++++++++++++++++ .../datasources/v2/OffloadDeltaCommand.scala | 6 +- .../spark/sql/delta/DeleteSQLSuite.scala | 65 ++++++ .../sql/delta/DeltaNativeWriteSuite.scala | 95 +++++++- .../GlutenDMLWithDeletionVectorsHelper.scala | 59 +++++ .../delta/commands/GlutenDeleteCommand.scala | 221 ++++++++++++++++++ .../spark/sql/delta/DeleteSQLSuite.scala | 63 +++++ .../sql/delta/DeltaNativeWriteSuite.scala | 97 ++++++++ 9 files changed, 884 insertions(+), 2 deletions(-) create mode 100644 backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/commands/GlutenDMLWithDeletionVectorsHelper.scala create mode 100644 backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala create mode 100644 backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDMLWithDeletionVectorsHelper.scala create mode 100644 backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala diff --git a/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/commands/GlutenDMLWithDeletionVectorsHelper.scala b/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/commands/GlutenDMLWithDeletionVectorsHelper.scala new file mode 100644 index 00000000000..2c7ce1ee6c7 --- /dev/null +++ b/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/commands/GlutenDMLWithDeletionVectorsHelper.scala @@ -0,0 +1,59 @@ +/* + * 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.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.delta.{DeltaLog, OptimisticTransaction} +import org.apache.spark.sql.delta.files.TahoeBatchFileIndex + +object GlutenDMLWithDeletionVectorsHelper extends DeltaCommand { + def findTouchedFiles( + sparkSession: SparkSession, + txn: OptimisticTransaction, + hasDVsEnabled: Boolean, + deltaLog: DeltaLog, + targetDf: DataFrame, + fileIndex: TahoeBatchFileIndex, + condition: Expression, + opName: String): Seq[TouchedFileWithDV] = { + require( + DMLWithDeletionVectorsHelper.SUPPORTED_DML_COMMANDS.contains(opName), + s"Expecting opName to be one of " + + s"${DMLWithDeletionVectorsHelper.SUPPORTED_DML_COMMANDS.mkString(", ")}, " + + s"but got '$opName'." + ) + + recordDeltaOperation(deltaLog, opType = s"$opName.findTouchedFiles.gluten") { + val candidateFiles = fileIndex.addFiles + val matchedRowIndexSets = + DeletionVectorBitmapGenerator.buildRowIndexSetsForFilesMatchingCondition( + sparkSession, + txn, + hasDVsEnabled, + targetDf, + candidateFiles, + condition) + + val nameToAddFileMap = generateCandidateFileMap(txn.deltaLog.dataPath, candidateFiles) + DMLWithDeletionVectorsHelper.findFilesWithMatchingRows( + txn, + nameToAddFileMap, + matchedRowIndexSets) + } + } +} diff --git a/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala b/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala new file mode 100644 index 00000000000..101efd67a1c --- /dev/null +++ b/backends-velox/src-delta33/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala @@ -0,0 +1,221 @@ +/* + * 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.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.delta.{DeltaLog, DeltaTableUtils, NumRecordsStats, OptimisticTransaction} +import org.apache.spark.sql.delta.actions.Action +import org.apache.spark.sql.delta.commands.MergeIntoCommandBase.totalBytesAndDistinctPartitionValues +import org.apache.spark.sql.delta.files.TahoeBatchFileIndex +import org.apache.spark.sql.delta.sources.DeltaSQLConf + +object GlutenDeleteCommand { + def apply(delegate: DeleteCommand): GlutenDeleteCommand = + new GlutenDeleteCommand( + delegate.deltaLog, + delegate.catalogTable, + delegate.target, + delegate.condition) + + def shouldOffload(delegate: DeleteCommand, sparkSession: SparkSession): Boolean = { + val persistentDeletionVectorsEnabled = + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS) + if (!persistentDeletionVectorsEnabled) { + return false + } + + delegate.condition.exists { + deleteCondition => + val snapshot = delegate.deltaLog.update() + val (_, dataPredicates) = DeltaTableUtils.splitMetadataAndDataPredicates( + deleteCondition, + snapshot.metadata.partitionColumns, + sparkSession) + dataPredicates.nonEmpty && DeletionVectorUtils.deletionVectorsWritable(snapshot) + } + } +} + +@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass")) +class GlutenDeleteCommand( + override val deltaLog: DeltaLog, + override val catalogTable: Option[CatalogTable], + override val target: LogicalPlan, + override val condition: Option[Expression]) + extends DeleteCommand(deltaLog, catalogTable, target, condition) { + + override def performDelete( + sparkSession: SparkSession, + deltaLog: DeltaLog, + txn: OptimisticTransaction): (Seq[Action], DeleteMetric) = { + val (cond, metadataPredicates, otherPredicates) = + condition match { + case Some(deleteCondition) => + val (metadata, data) = DeltaTableUtils.splitMetadataAndDataPredicates( + deleteCondition, + txn.metadata.partitionColumns, + sparkSession) + (deleteCondition, metadata, data) + case None => + return super.performDelete(sparkSession, deltaLog, txn) + } + + val shouldWriteDVs = otherPredicates.nonEmpty && + shouldWritePersistentDeletionVectors(sparkSession, txn) + if (!shouldWriteDVs) { + return super.performDelete(sparkSession, deltaLog, txn) + } + + var numRemovedFiles: Long = 0 + val numAddedFiles: Long = 0 + var scanTimeMs: Long = 0 + val rewriteTimeMs: Long = 0 + val numAddedBytes: Long = 0 + val changeFileBytes: Long = 0 + val numRemovedBytes: Long = 0 + val numFilesBeforeSkipping: Long = txn.snapshot.numOfFiles + val numBytesBeforeSkipping: Long = txn.snapshot.sizeInBytes + var numFilesAfterSkipping: Long = 0 + var numBytesAfterSkipping: Long = 0 + var numPartitionsAfterSkipping: Option[Long] = None + val numPartitionsRemovedFrom: Option[Long] = None + val numPartitionsAddedTo: Option[Long] = None + var numDeletedRows: Option[Long] = None + val numCopiedRows: Option[Long] = None + var numDeletionVectorsAdded: Long = 0 + var numDeletionVectorsRemoved: Long = 0 + var numDeletionVectorsUpdated: Long = 0 + + val startTime = System.nanoTime() + val numFilesTotal = txn.snapshot.numOfFiles + val candidateFiles = txn.filterFiles( + metadataPredicates ++ otherPredicates, + keepNumRecords = true) + + numFilesAfterSkipping = candidateFiles.size + val (numCandidateBytes, numCandidatePartitions) = + totalBytesAndDistinctPartitionValues(candidateFiles) + numBytesAfterSkipping = numCandidateBytes + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsAfterSkipping = Some(numCandidatePartitions) + } + + val fileIndex = new TahoeBatchFileIndex( + sparkSession, + "delete", + candidateFiles, + deltaLog, + deltaLog.dataPath, + txn.snapshot) + val targetDf = DMLWithDeletionVectorsHelper.createTargetDfForScanningForMatches( + sparkSession, + target, + fileIndex) + val mustReadDeletionVectors = DeletionVectorUtils.deletionVectorsReadable(txn.snapshot) + val touchedFiles = GlutenDMLWithDeletionVectorsHelper.findTouchedFiles( + sparkSession, + txn, + mustReadDeletionVectors, + deltaLog, + targetDf, + fileIndex, + cond, + opName = "DELETE") + + scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + val deleteActions = + if (touchedFiles.nonEmpty) { + val (actions, metricMap) = DMLWithDeletionVectorsHelper.processUnmodifiedData( + sparkSession, + touchedFiles, + txn.snapshot) + metrics("numDeletedRows").set(metricMap("numModifiedRows")) + numDeletedRows = Some(metricMap("numModifiedRows")) + numDeletionVectorsAdded = metricMap("numDeletionVectorsAdded") + numDeletionVectorsRemoved = metricMap("numDeletionVectorsRemoved") + numDeletionVectorsUpdated = metricMap("numDeletionVectorsUpdated") + numRemovedFiles = metricMap("numRemovedFiles") + actions + } else { + Nil + } + + metrics("numRemovedFiles").set(numRemovedFiles) + metrics("numAddedFiles").set(numAddedFiles) + val executionTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + metrics("executionTimeMs").set(executionTimeMs) + metrics("scanTimeMs").set(scanTimeMs) + metrics("rewriteTimeMs").set(rewriteTimeMs) + metrics("numAddedChangeFiles").set(0L) + metrics("changeFileBytes").set(changeFileBytes) + metrics("numAddedBytes").set(numAddedBytes) + metrics("numRemovedBytes").set(numRemovedBytes) + metrics("numFilesBeforeSkipping").set(numFilesBeforeSkipping) + metrics("numBytesBeforeSkipping").set(numBytesBeforeSkipping) + metrics("numFilesAfterSkipping").set(numFilesAfterSkipping) + metrics("numBytesAfterSkipping").set(numBytesAfterSkipping) + metrics("numDeletionVectorsAdded").set(numDeletionVectorsAdded) + metrics("numDeletionVectorsRemoved").set(numDeletionVectorsRemoved) + metrics("numDeletionVectorsUpdated").set(numDeletionVectorsUpdated) + numPartitionsAfterSkipping.foreach(metrics("numPartitionsAfterSkipping").set) + numPartitionsAddedTo.foreach(metrics("numPartitionsAddedTo").set) + numPartitionsRemovedFrom.foreach(metrics("numPartitionsRemovedFrom").set) + numCopiedRows.foreach(metrics("numCopiedRows").set) + txn.registerSQLMetrics(sparkSession, metrics) + sendDriverMetrics(sparkSession, metrics) + + val numRecordsStats = NumRecordsStats.fromActions(deleteActions) + val deleteMetric = DeleteMetric( + condition = condition.map(_.sql).getOrElse("true"), + numFilesTotal, + numFilesAfterSkipping, + numAddedFiles, + numRemovedFiles, + numAddedFiles, + numAddedChangeFiles = 0L, + numFilesBeforeSkipping, + numBytesBeforeSkipping, + numFilesAfterSkipping, + numBytesAfterSkipping, + numPartitionsAfterSkipping, + numPartitionsAddedTo, + numPartitionsRemovedFrom, + numCopiedRows, + numDeletedRows, + numAddedBytes, + numRemovedBytes, + changeFileBytes = changeFileBytes, + scanTimeMs, + rewriteTimeMs, + numDeletionVectorsAdded, + numDeletionVectorsRemoved, + numDeletionVectorsUpdated, + numLogicalRecordsAdded = numRecordsStats.numLogicalRecordsAdded, + numLogicalRecordsRemoved = numRecordsStats.numLogicalRecordsRemoved + ) + + val actionsToCommit = if (deleteActions.nonEmpty) { + createSetTransaction(sparkSession, deltaLog).toSeq ++ deleteActions + } else { + Seq.empty + } + (actionsToCommit, deleteMetric) + } +} diff --git a/backends-velox/src-delta33/main/scala/org/apache/spark/sql/execution/datasources/v2/OffloadDeltaCommand.scala b/backends-velox/src-delta33/main/scala/org/apache/spark/sql/execution/datasources/v2/OffloadDeltaCommand.scala index f4b74e4dbec..39969f42dbf 100644 --- a/backends-velox/src-delta33/main/scala/org/apache/spark/sql/execution/datasources/v2/OffloadDeltaCommand.scala +++ b/backends-velox/src-delta33/main/scala/org/apache/spark/sql/execution/datasources/v2/OffloadDeltaCommand.scala @@ -19,8 +19,9 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.gluten.config.VeloxDeltaConfig import org.apache.gluten.extension.columnar.offload.OffloadSingleNode +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.catalog.DeltaCatalog -import org.apache.spark.sql.delta.commands.{DeleteCommand, DeltaCommand, OptimizeTableCommand, UpdateCommand} +import org.apache.spark.sql.delta.commands.{DeleteCommand, DeltaCommand, GlutenDeleteCommand, OptimizeTableCommand, UpdateCommand} import org.apache.spark.sql.delta.skipping.clustering.ClusteredTableUtils import org.apache.spark.sql.delta.sources.DeltaDataSource import org.apache.spark.sql.execution.SparkPlan @@ -35,6 +36,9 @@ case class OffloadDeltaCommand() extends OffloadSingleNode with DeltaCommand { plan match { case ExecutedCommandExec(uc: UpdateCommand) => ExecutedCommandExec(GlutenDeltaLeafRunnableCommand(uc)) + case ExecutedCommandExec(dc: DeleteCommand) + if GlutenDeleteCommand.shouldOffload(dc, SparkSession.active) => + ExecutedCommandExec(GlutenDeltaLeafRunnableCommand(GlutenDeleteCommand(dc))) case ExecutedCommandExec(dc: DeleteCommand) => ExecutedCommandExec(GlutenDeltaLeafRunnableCommand(dc)) case ExecutedCommandExec(optimize: OptimizeTableCommand) if shouldOffloadOptimize(optimize) => diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala index b43e7945dcf..9a7317df8af 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala @@ -143,6 +143,71 @@ class DeleteSQLWithDeletionVectorsSuite text = "SELECT key, value, 1 FROM tab", expectResult = Row(0, 3, 1) :: Nil) } + + test("repeated DELETE produces, updates, and removes persistent deletion vectors") { + withTempDir { + dir => + val path = dir.getCanonicalPath + spark.range(0, 10, 1, numPartitions = 1).toDF("id").write.format("delta").save(path) + val log = DeltaLog.forTable(spark, path) + + def assertRows(expected: Long*): Unit = { + checkAnswer( + sql(s"SELECT id FROM delta.`$path` ORDER BY id"), + expected.map(id => Row(id))) + } + + def assertActiveDeletionVectors(expectedFiles: Int, expectedCardinality: Long): Unit = { + val filesWithDVs = getFilesWithDeletionVectors(log) + assert(filesWithDVs.size === expectedFiles) + assert(filesWithDVs.map(_.deletionVector.cardinality).sum === expectedCardinality) + } + + def assertDeleteMetrics(expected: (String, Long)*): Unit = { + val metrics = io.delta.tables.DeltaTable + .forPath(path) + .history() + .select("operationMetrics") + .take(1) + .head + .getMap(0) + .asInstanceOf[Map[String, String]] + .map { case (key, value) => key -> value.toLong } + expected.foreach { + case (key, value) => + assert(metrics.getOrElse(key, -1L) === value, s"Unexpected metric $key: $metrics") + } + } + + executeDelete(s"delta.`$path`", "id % 3 = 0") + assertRows(1, 2, 4, 5, 7, 8) + assertActiveDeletionVectors(expectedFiles = 1, expectedCardinality = 4) + assertDeleteMetrics( + "numDeletedRows" -> 4L, + "numDeletionVectorsAdded" -> 1L, + "numDeletionVectorsUpdated" -> 0L, + "numDeletionVectorsRemoved" -> 0L) + + executeDelete(s"delta.`$path`", "id IN (4, 5, 7)") + assertRows(1, 2, 8) + assertActiveDeletionVectors(expectedFiles = 1, expectedCardinality = 7) + assertDeleteMetrics( + "numDeletedRows" -> 3L, + "numDeletionVectorsAdded" -> 0L, + "numDeletionVectorsUpdated" -> 1L, + "numDeletionVectorsRemoved" -> 0L) + + executeDelete(s"delta.`$path`", "id IN (1, 2, 8)") + assertRows() + assertActiveDeletionVectors(expectedFiles = 0, expectedCardinality = 0) + assertDeleteMetrics( + "numDeletedRows" -> 3L, + "numRemovedFiles" -> 1L, + "numDeletionVectorsAdded" -> 0L, + "numDeletionVectorsUpdated" -> 0L, + "numDeletionVectorsRemoved" -> 1L) + } + } } @ExtendedSQLTest diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala index 26205711869..68c2c8f579b 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala @@ -19,9 +19,10 @@ package org.apache.spark.sql.delta import org.apache.gluten.config.GlutenConfig import org.apache.gluten.config.VeloxDeltaConfig -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.commands.GlutenDeleteCommand import org.apache.spark.sql.delta.commands.optimize.OptimizeMetrics import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaSQLCommandTest @@ -91,6 +92,26 @@ class DeltaNativeWriteSuite extends DeltaSQLCommandTest { ) } + private def hasGlutenDeleteCommand(plan: SparkPlan): Boolean = { + plan + .collectFirst { + case ExecutedCommandExec(GlutenDeltaLeafRunnableCommand(_: GlutenDeleteCommand)) => true + } + .getOrElse(false) + } + + private def assertContainsGlutenDeleteCommand(plan: SparkPlan, context: String): Unit = { + assert( + hasGlutenDeleteCommand(plan), + s"Expected GlutenDeleteCommand for $context, but got plan:\n${plan.treeString}") + } + + private def assertNoGlutenDeleteCommand(plan: SparkPlan, context: String): Unit = { + assert( + !hasGlutenDeleteCommand(plan), + s"Expected no GlutenDeleteCommand for $context, but got plan:\n${plan.treeString}") + } + private def files(deltaLog: DeltaLog): Set[AddFile] = { deltaLog.update().allFiles.collect().toSet } @@ -284,4 +305,76 @@ class DeltaNativeWriteSuite extends DeltaSQLCommandTest { } } } + + test("DELETE command route is limited to persistent DV row-condition deletes") { + withNativeWriteOffloadConf { + withTempDir { + dir => + val path = dir.getCanonicalPath + Seq((1, "a"), (2, "b")).toDF("id", "value").write.format("delta").save(path) + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + + withSQLConf(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true") { + val deleteDf = sql(s"DELETE FROM delta.`$path` WHERE id = 1") + assertContainsGlutenDeleteCommand( + deleteDf.queryExecution.executedPlan, + "persistent DV row-condition DELETE") + deleteDf.collect() + } + assert(spark.read.format("delta").load(path).collect().toSet == Set(Row(2, "b"))) + } + + withTempDir { + dir => + val path = dir.getCanonicalPath + Seq((1, "a"), (2, "b")).toDF("id", "value").write.format("delta").save(path) + + val deleteDf = sql(s"DELETE FROM delta.`$path` WHERE id = 1") + assertNoGlutenDeleteCommand(deleteDf.queryExecution.executedPlan, "ordinary DELETE") + deleteDf.collect() + assert(spark.read.format("delta").load(path).collect().toSet == Set(Row(2, "b"))) + } + + withTempDir { + dir => + val path = dir.getCanonicalPath + spark + .range(0, 4) + .selectExpr("id", "cast(id % 2 as int) as part") + .write + .format("delta") + .partitionBy("part") + .save(path) + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + + withSQLConf(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true") { + val deleteDf = sql(s"DELETE FROM delta.`$path` WHERE part = 0") + assertNoGlutenDeleteCommand( + deleteDf.queryExecution.executedPlan, + "metadata-only DELETE") + deleteDf.collect() + } + assert( + spark.read.format("delta").load(path).select("id").collect().map(_.getLong(0)).toSet == + Set(1L, 3L)) + } + + withTempDir { + dir => + val path = dir.getCanonicalPath + Seq((1, "a"), (2, "b")).toDF("id", "value").write.format("delta").save(path) + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + + withSQLConf(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true") { + val deleteDf = sql(s"DELETE FROM delta.`$path`") + assertNoGlutenDeleteCommand(deleteDf.queryExecution.executedPlan, "full-table DELETE") + deleteDf.collect() + } + assert(spark.read.format("delta").load(path).count() == 0) + } + } + } } diff --git a/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDMLWithDeletionVectorsHelper.scala b/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDMLWithDeletionVectorsHelper.scala new file mode 100644 index 00000000000..2c7ce1ee6c7 --- /dev/null +++ b/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDMLWithDeletionVectorsHelper.scala @@ -0,0 +1,59 @@ +/* + * 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.sql.{DataFrame, SparkSession} +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.delta.{DeltaLog, OptimisticTransaction} +import org.apache.spark.sql.delta.files.TahoeBatchFileIndex + +object GlutenDMLWithDeletionVectorsHelper extends DeltaCommand { + def findTouchedFiles( + sparkSession: SparkSession, + txn: OptimisticTransaction, + hasDVsEnabled: Boolean, + deltaLog: DeltaLog, + targetDf: DataFrame, + fileIndex: TahoeBatchFileIndex, + condition: Expression, + opName: String): Seq[TouchedFileWithDV] = { + require( + DMLWithDeletionVectorsHelper.SUPPORTED_DML_COMMANDS.contains(opName), + s"Expecting opName to be one of " + + s"${DMLWithDeletionVectorsHelper.SUPPORTED_DML_COMMANDS.mkString(", ")}, " + + s"but got '$opName'." + ) + + recordDeltaOperation(deltaLog, opType = s"$opName.findTouchedFiles.gluten") { + val candidateFiles = fileIndex.addFiles + val matchedRowIndexSets = + DeletionVectorBitmapGenerator.buildRowIndexSetsForFilesMatchingCondition( + sparkSession, + txn, + hasDVsEnabled, + targetDf, + candidateFiles, + condition) + + val nameToAddFileMap = generateCandidateFileMap(txn.deltaLog.dataPath, candidateFiles) + DMLWithDeletionVectorsHelper.findFilesWithMatchingRows( + txn, + nameToAddFileMap, + matchedRowIndexSets) + } + } +} diff --git a/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala b/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala new file mode 100644 index 00000000000..101efd67a1c --- /dev/null +++ b/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala @@ -0,0 +1,221 @@ +/* + * 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.sql.SparkSession +import org.apache.spark.sql.catalyst.catalog.CatalogTable +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.delta.{DeltaLog, DeltaTableUtils, NumRecordsStats, OptimisticTransaction} +import org.apache.spark.sql.delta.actions.Action +import org.apache.spark.sql.delta.commands.MergeIntoCommandBase.totalBytesAndDistinctPartitionValues +import org.apache.spark.sql.delta.files.TahoeBatchFileIndex +import org.apache.spark.sql.delta.sources.DeltaSQLConf + +object GlutenDeleteCommand { + def apply(delegate: DeleteCommand): GlutenDeleteCommand = + new GlutenDeleteCommand( + delegate.deltaLog, + delegate.catalogTable, + delegate.target, + delegate.condition) + + def shouldOffload(delegate: DeleteCommand, sparkSession: SparkSession): Boolean = { + val persistentDeletionVectorsEnabled = + sparkSession.sessionState.conf.getConf(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS) + if (!persistentDeletionVectorsEnabled) { + return false + } + + delegate.condition.exists { + deleteCondition => + val snapshot = delegate.deltaLog.update() + val (_, dataPredicates) = DeltaTableUtils.splitMetadataAndDataPredicates( + deleteCondition, + snapshot.metadata.partitionColumns, + sparkSession) + dataPredicates.nonEmpty && DeletionVectorUtils.deletionVectorsWritable(snapshot) + } + } +} + +@SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass")) +class GlutenDeleteCommand( + override val deltaLog: DeltaLog, + override val catalogTable: Option[CatalogTable], + override val target: LogicalPlan, + override val condition: Option[Expression]) + extends DeleteCommand(deltaLog, catalogTable, target, condition) { + + override def performDelete( + sparkSession: SparkSession, + deltaLog: DeltaLog, + txn: OptimisticTransaction): (Seq[Action], DeleteMetric) = { + val (cond, metadataPredicates, otherPredicates) = + condition match { + case Some(deleteCondition) => + val (metadata, data) = DeltaTableUtils.splitMetadataAndDataPredicates( + deleteCondition, + txn.metadata.partitionColumns, + sparkSession) + (deleteCondition, metadata, data) + case None => + return super.performDelete(sparkSession, deltaLog, txn) + } + + val shouldWriteDVs = otherPredicates.nonEmpty && + shouldWritePersistentDeletionVectors(sparkSession, txn) + if (!shouldWriteDVs) { + return super.performDelete(sparkSession, deltaLog, txn) + } + + var numRemovedFiles: Long = 0 + val numAddedFiles: Long = 0 + var scanTimeMs: Long = 0 + val rewriteTimeMs: Long = 0 + val numAddedBytes: Long = 0 + val changeFileBytes: Long = 0 + val numRemovedBytes: Long = 0 + val numFilesBeforeSkipping: Long = txn.snapshot.numOfFiles + val numBytesBeforeSkipping: Long = txn.snapshot.sizeInBytes + var numFilesAfterSkipping: Long = 0 + var numBytesAfterSkipping: Long = 0 + var numPartitionsAfterSkipping: Option[Long] = None + val numPartitionsRemovedFrom: Option[Long] = None + val numPartitionsAddedTo: Option[Long] = None + var numDeletedRows: Option[Long] = None + val numCopiedRows: Option[Long] = None + var numDeletionVectorsAdded: Long = 0 + var numDeletionVectorsRemoved: Long = 0 + var numDeletionVectorsUpdated: Long = 0 + + val startTime = System.nanoTime() + val numFilesTotal = txn.snapshot.numOfFiles + val candidateFiles = txn.filterFiles( + metadataPredicates ++ otherPredicates, + keepNumRecords = true) + + numFilesAfterSkipping = candidateFiles.size + val (numCandidateBytes, numCandidatePartitions) = + totalBytesAndDistinctPartitionValues(candidateFiles) + numBytesAfterSkipping = numCandidateBytes + if (txn.metadata.partitionColumns.nonEmpty) { + numPartitionsAfterSkipping = Some(numCandidatePartitions) + } + + val fileIndex = new TahoeBatchFileIndex( + sparkSession, + "delete", + candidateFiles, + deltaLog, + deltaLog.dataPath, + txn.snapshot) + val targetDf = DMLWithDeletionVectorsHelper.createTargetDfForScanningForMatches( + sparkSession, + target, + fileIndex) + val mustReadDeletionVectors = DeletionVectorUtils.deletionVectorsReadable(txn.snapshot) + val touchedFiles = GlutenDMLWithDeletionVectorsHelper.findTouchedFiles( + sparkSession, + txn, + mustReadDeletionVectors, + deltaLog, + targetDf, + fileIndex, + cond, + opName = "DELETE") + + scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + val deleteActions = + if (touchedFiles.nonEmpty) { + val (actions, metricMap) = DMLWithDeletionVectorsHelper.processUnmodifiedData( + sparkSession, + touchedFiles, + txn.snapshot) + metrics("numDeletedRows").set(metricMap("numModifiedRows")) + numDeletedRows = Some(metricMap("numModifiedRows")) + numDeletionVectorsAdded = metricMap("numDeletionVectorsAdded") + numDeletionVectorsRemoved = metricMap("numDeletionVectorsRemoved") + numDeletionVectorsUpdated = metricMap("numDeletionVectorsUpdated") + numRemovedFiles = metricMap("numRemovedFiles") + actions + } else { + Nil + } + + metrics("numRemovedFiles").set(numRemovedFiles) + metrics("numAddedFiles").set(numAddedFiles) + val executionTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 + metrics("executionTimeMs").set(executionTimeMs) + metrics("scanTimeMs").set(scanTimeMs) + metrics("rewriteTimeMs").set(rewriteTimeMs) + metrics("numAddedChangeFiles").set(0L) + metrics("changeFileBytes").set(changeFileBytes) + metrics("numAddedBytes").set(numAddedBytes) + metrics("numRemovedBytes").set(numRemovedBytes) + metrics("numFilesBeforeSkipping").set(numFilesBeforeSkipping) + metrics("numBytesBeforeSkipping").set(numBytesBeforeSkipping) + metrics("numFilesAfterSkipping").set(numFilesAfterSkipping) + metrics("numBytesAfterSkipping").set(numBytesAfterSkipping) + metrics("numDeletionVectorsAdded").set(numDeletionVectorsAdded) + metrics("numDeletionVectorsRemoved").set(numDeletionVectorsRemoved) + metrics("numDeletionVectorsUpdated").set(numDeletionVectorsUpdated) + numPartitionsAfterSkipping.foreach(metrics("numPartitionsAfterSkipping").set) + numPartitionsAddedTo.foreach(metrics("numPartitionsAddedTo").set) + numPartitionsRemovedFrom.foreach(metrics("numPartitionsRemovedFrom").set) + numCopiedRows.foreach(metrics("numCopiedRows").set) + txn.registerSQLMetrics(sparkSession, metrics) + sendDriverMetrics(sparkSession, metrics) + + val numRecordsStats = NumRecordsStats.fromActions(deleteActions) + val deleteMetric = DeleteMetric( + condition = condition.map(_.sql).getOrElse("true"), + numFilesTotal, + numFilesAfterSkipping, + numAddedFiles, + numRemovedFiles, + numAddedFiles, + numAddedChangeFiles = 0L, + numFilesBeforeSkipping, + numBytesBeforeSkipping, + numFilesAfterSkipping, + numBytesAfterSkipping, + numPartitionsAfterSkipping, + numPartitionsAddedTo, + numPartitionsRemovedFrom, + numCopiedRows, + numDeletedRows, + numAddedBytes, + numRemovedBytes, + changeFileBytes = changeFileBytes, + scanTimeMs, + rewriteTimeMs, + numDeletionVectorsAdded, + numDeletionVectorsRemoved, + numDeletionVectorsUpdated, + numLogicalRecordsAdded = numRecordsStats.numLogicalRecordsAdded, + numLogicalRecordsRemoved = numRecordsStats.numLogicalRecordsRemoved + ) + + val actionsToCommit = if (deleteActions.nonEmpty) { + createSetTransaction(sparkSession, deltaLog).toSeq ++ deleteActions + } else { + Seq.empty + } + (actionsToCommit, deleteMetric) + } +} diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala index 54fdeac8f2f..275f3b6c6f4 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala @@ -144,6 +144,69 @@ class DeleteSQLWithDeletionVectorsSuite extends DeleteSQLSuite text = "SELECT key, value, 1 FROM tab", expectResult = Row(0, 3, 1) :: Nil) } + + test("repeated DELETE produces, updates, and removes persistent deletion vectors") { + withTempDir { dir => + val path = dir.getCanonicalPath + spark.range(0, 10, 1, numPartitions = 1).toDF("id").write.format("delta").save(path) + val log = DeltaLog.forTable(spark, path) + + def assertRows(expected: Long*): Unit = { + checkAnswer( + sql(s"SELECT id FROM delta.`$path` ORDER BY id"), + expected.map(id => Row(id))) + } + + def assertActiveDeletionVectors(expectedFiles: Int, expectedCardinality: Long): Unit = { + val filesWithDVs = getFilesWithDeletionVectors(log) + assert(filesWithDVs.size === expectedFiles) + assert(filesWithDVs.map(_.deletionVector.cardinality).sum === expectedCardinality) + } + + def assertDeleteMetrics(expected: (String, Long)*): Unit = { + val metrics = io.delta.tables.DeltaTable + .forPath(path) + .history() + .select("operationMetrics") + .take(1) + .head + .getMap(0) + .asInstanceOf[Map[String, String]] + .map { case (key, value) => key -> value.toLong } + expected.foreach { case (key, value) => + assert(metrics.getOrElse(key, -1L) === value, s"Unexpected metric $key: $metrics") + } + } + + executeDelete(s"delta.`$path`", "id % 3 = 0") + assertRows(1, 2, 4, 5, 7, 8) + assertActiveDeletionVectors(expectedFiles = 1, expectedCardinality = 4) + assertDeleteMetrics( + "numDeletedRows" -> 4L, + "numDeletionVectorsAdded" -> 1L, + "numDeletionVectorsUpdated" -> 0L, + "numDeletionVectorsRemoved" -> 0L) + + executeDelete(s"delta.`$path`", "id IN (4, 5, 7)") + assertRows(1, 2, 8) + assertActiveDeletionVectors(expectedFiles = 1, expectedCardinality = 7) + assertDeleteMetrics( + "numDeletedRows" -> 3L, + "numDeletionVectorsAdded" -> 0L, + "numDeletionVectorsUpdated" -> 1L, + "numDeletionVectorsRemoved" -> 0L) + + executeDelete(s"delta.`$path`", "id IN (1, 2, 8)") + assertRows() + assertActiveDeletionVectors(expectedFiles = 0, expectedCardinality = 0) + assertDeleteMetrics( + "numDeletedRows" -> 3L, + "numRemovedFiles" -> 1L, + "numDeletionVectorsAdded" -> 0L, + "numDeletionVectorsUpdated" -> 0L, + "numDeletionVectorsRemoved" -> 1L) + } + } } @ExtendedSQLTest diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala index bca0a66d1ad..70b29a7d34c 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala @@ -22,6 +22,7 @@ import org.apache.gluten.config.VeloxDeltaConfig import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.delta.actions.AddFile +import org.apache.spark.sql.delta.commands.GlutenDeleteCommand import org.apache.spark.sql.delta.commands.optimize.OptimizeMetrics import org.apache.spark.sql.delta.sources.DeltaSQLConf import org.apache.spark.sql.delta.test.DeltaSQLCommandTest @@ -156,6 +157,28 @@ class DeltaNativeWriteSuite extends DeltaSQLCommandTest { ) } + private def hasGlutenDeleteCommand(plan: SparkPlan): Boolean = { + plan + .collectFirst { + case ExecutedCommandExec(GlutenDeltaLeafRunnableCommand(_: GlutenDeleteCommand)) => true + } + .getOrElse(false) + } + + private def assertContainsGlutenDeleteCommand(plans: Seq[SparkPlan], context: String): Unit = { + assert( + plans.exists(hasGlutenDeleteCommand), + s"Expected GlutenDeleteCommand for $context, but got plans:\n" + + plans.map(_.treeString).mkString("\n---\n")) + } + + private def assertNoGlutenDeleteCommand(plans: Seq[SparkPlan], context: String): Unit = { + assert( + !plans.exists(hasGlutenDeleteCommand), + s"Expected no GlutenDeleteCommand for $context, but got plans:\n" + + plans.map(_.treeString).mkString("\n---\n")) + } + private def files(deltaLog: DeltaLog): Set[AddFile] = { deltaLog.update().allFiles.collect().toSet } @@ -242,6 +265,80 @@ class DeltaNativeWriteSuite extends DeltaSQLCommandTest { } } + test("DELETE command route is limited to persistent DV row-condition deletes") { + withNativeWriteOffloadConf { + withTempDir { + dir => + val path = dir.getCanonicalPath + Seq((1, "a"), (2, "b")).toDF("id", "value").write.format("delta").save(path) + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + + withSQLConf(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true") { + val deleteDf = sql(s"DELETE FROM delta.`$path` WHERE id = 1") + assertContainsGlutenDeleteCommand( + Seq(deleteDf.queryExecution.executedPlan), + "persistent DV row-condition DELETE") + deleteDf.collect() + } + assert(spark.read.format("delta").load(path).collect().toSet == Set(Row(2, "b"))) + } + + withTempDir { + dir => + val path = dir.getCanonicalPath + Seq((1, "a"), (2, "b")).toDF("id", "value").write.format("delta").save(path) + + val deleteDf = sql(s"DELETE FROM delta.`$path` WHERE id = 1") + assertNoGlutenDeleteCommand(Seq(deleteDf.queryExecution.executedPlan), "ordinary DELETE") + deleteDf.collect() + assert(spark.read.format("delta").load(path).collect().toSet == Set(Row(2, "b"))) + } + + withTempDir { + dir => + val path = dir.getCanonicalPath + spark + .range(0, 4) + .selectExpr("id", "cast(id % 2 as int) as part") + .write + .format("delta") + .partitionBy("part") + .save(path) + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + + withSQLConf(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true") { + val deleteDf = sql(s"DELETE FROM delta.`$path` WHERE part = 0") + assertNoGlutenDeleteCommand( + Seq(deleteDf.queryExecution.executedPlan), + "metadata-only DELETE") + deleteDf.collect() + } + assert( + spark.read.format("delta").load(path).select("id").collect().map(_.getLong(0)).toSet == + Set(1L, 3L)) + } + + withTempDir { + dir => + val path = dir.getCanonicalPath + Seq((1, "a"), (2, "b")).toDF("id", "value").write.format("delta").save(path) + spark.sql( + s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") + + withSQLConf(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true") { + val deleteDf = sql(s"DELETE FROM delta.`$path`") + assertNoGlutenDeleteCommand( + Seq(deleteDf.queryExecution.executedPlan), + "full-table DELETE") + deleteDf.collect() + } + assert(spark.read.format("delta").load(path).count() == 0) + } + } + } + test("native delta CTAS command should be offloaded") { withNativeWriteOffloadConf { withTable("delta_native_write_ctas") { From 3ba36725ce1c9ee7f8d8c2335b3d255402a9c2f2 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 17:29:08 +0300 Subject: [PATCH 10/16] [VL][Delta] Adapt DELETE DV helper call for Delta 4.1 --- .../delta/commands/GlutenDeleteCommand.scala | 55 ++++++++++++++++++- 1 file changed, 52 insertions(+), 3 deletions(-) diff --git a/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala b/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala index 101efd67a1c..9c18a25ddeb 100644 --- a/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala +++ b/backends-velox/src-delta40/main/scala/org/apache/spark/sql/delta/commands/GlutenDeleteCommand.scala @@ -21,12 +21,14 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.delta.{DeltaLog, DeltaTableUtils, NumRecordsStats, OptimisticTransaction} -import org.apache.spark.sql.delta.actions.Action +import org.apache.spark.sql.delta.actions.{Action, Metadata} import org.apache.spark.sql.delta.commands.MergeIntoCommandBase.totalBytesAndDistinctPartitionValues import org.apache.spark.sql.delta.files.TahoeBatchFileIndex import org.apache.spark.sql.delta.sources.DeltaSQLConf object GlutenDeleteCommand { + private val processUnmodifiedDataMethodName = "processUnmodifiedData" + def apply(delegate: DeleteCommand): GlutenDeleteCommand = new GlutenDeleteCommand( delegate.deltaLog, @@ -51,6 +53,53 @@ object GlutenDeleteCommand { dataPredicates.nonEmpty && DeletionVectorUtils.deletionVectorsWritable(snapshot) } } + + private def processUnmodifiedData( + sparkSession: SparkSession, + touchedFiles: Seq[TouchedFileWithDV], + txn: OptimisticTransaction): (Seq[Action], Map[String, Long]) = { + val helper = DMLWithDeletionVectorsHelper + val method = helper.getClass.getMethods.find { + method => method.getName == processUnmodifiedDataMethodName && method.getParameterCount == 4 + }.getOrElse { + helper.getClass.getMethods.find { + method => method.getName == processUnmodifiedDataMethodName && method.getParameterCount == 3 + }.getOrElse { + throw new IllegalStateException( + s"Unable to find $processUnmodifiedDataMethodName on ${helper.getClass.getName}") + } + } + + val result = + if (method.getParameterCount == 4) { + method.invoke( + helper, + sparkSession, + touchedFiles, + txn.snapshot, + Int.box(dataSkippingStringPrefixLength(sparkSession, txn.metadata))) + } else { + method.invoke( + helper, + sparkSession, + touchedFiles, + txn.snapshot) + } + + result.asInstanceOf[(Seq[Action], Map[String, Long])] + } + + private def dataSkippingStringPrefixLength( + sparkSession: SparkSession, + metadata: Metadata): Int = { + val statsCollectionUtilsClass = + Class.forName("org.apache.spark.sql.delta.stats.StatsCollectionUtils$") + val statsCollectionUtils = statsCollectionUtilsClass.getField("MODULE$").get(null) + statsCollectionUtilsClass + .getMethod("getDataSkippingStringPrefixLength", classOf[SparkSession], classOf[Metadata]) + .invoke(statsCollectionUtils, sparkSession, metadata) + .asInstanceOf[Int] + } } @SuppressWarnings(Array("io.github.zhztheplayer.scalawarts.InheritFromCaseClass")) @@ -142,10 +191,10 @@ class GlutenDeleteCommand( scanTimeMs = (System.nanoTime() - startTime) / 1000 / 1000 val deleteActions = if (touchedFiles.nonEmpty) { - val (actions, metricMap) = DMLWithDeletionVectorsHelper.processUnmodifiedData( + val (actions, metricMap) = GlutenDeleteCommand.processUnmodifiedData( sparkSession, touchedFiles, - txn.snapshot) + txn) metrics("numDeletedRows").set(metricMap("numModifiedRows")) numDeletedRows = Some(metricMap("numModifiedRows")) numDeletionVectorsAdded = metricMap("numDeletionVectorsAdded") From 314055afab7c4e5fb43f72dae942d9a9d7309976 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 19:03:35 +0300 Subject: [PATCH 11/16] [VL][Delta] Stabilize DELETE DV CI assertions --- .../sql/delta/DeltaNativeWriteSuite.scala | 10 ++++++++- .../sql/delta/DeltaNativeWriteSuite.scala | 10 ++++++++- .../apache/gluten/execution/DeltaSuite.scala | 22 ++++++++++--------- 3 files changed, 30 insertions(+), 12 deletions(-) diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala index 68c2c8f579b..ae21ea39036 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala @@ -93,11 +93,19 @@ class DeltaNativeWriteSuite extends DeltaSQLCommandTest { } private def hasGlutenDeleteCommand(plan: SparkPlan): Boolean = { - plan + val commandClassMatch = plan .collectFirst { case ExecutedCommandExec(GlutenDeltaLeafRunnableCommand(_: GlutenDeleteCommand)) => true } .getOrElse(false) + + val commandNodeMatch = plan + .collectFirst { + case p if p.nodeName.contains("GlutenDeleteCommand") => true + } + .getOrElse(false) + + commandClassMatch || commandNodeMatch || plan.treeString.contains("GlutenDeleteCommand") } private def assertContainsGlutenDeleteCommand(plan: SparkPlan, context: String): Unit = { diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala index 70b29a7d34c..c590b3efde4 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaNativeWriteSuite.scala @@ -158,11 +158,19 @@ class DeltaNativeWriteSuite extends DeltaSQLCommandTest { } private def hasGlutenDeleteCommand(plan: SparkPlan): Boolean = { - plan + val commandClassMatch = plan .collectFirst { case ExecutedCommandExec(GlutenDeltaLeafRunnableCommand(_: GlutenDeleteCommand)) => true } .getOrElse(false) + + val commandNodeMatch = plan + .collectFirst { + case p if p.nodeName.contains("GlutenDeleteCommand") => true + } + .getOrElse(false) + + commandClassMatch || commandNodeMatch || plan.treeString.contains("GlutenDeleteCommand") } private def assertContainsGlutenDeleteCommand(plans: Seq[SparkPlan], context: String): Unit = { diff --git a/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala b/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala index d7f9e66e579..a9a3cc87a0a 100644 --- a/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala +++ b/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala @@ -210,18 +210,20 @@ abstract class DeltaSuite extends WholeStageTransformerSuite { spark.sql( s"ALTER TABLE delta.`$path` SET TBLPROPERTIES ('delta.enableDeletionVectors' = true)") checkAnswer(spark.read.format("delta").load(path), df1.union(df2)) - spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (${values2.mkString(", ")})") - val df = spark.read.format("delta").load(path) - val executedPlan = df.queryExecution.executedPlan - if (SparkVersionUtil.gteSpark35) { - assert(executedPlan.collect { case _: DeltaScanTransformer => true }.nonEmpty) + withSQLConf("spark.databricks.delta.deletionVectors.useMetadataRowIndex" -> "true") { + spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (${values2.mkString(", ")})") + val df = spark.read.format("delta").load(path) + val executedPlan = df.queryExecution.executedPlan val planText = executedPlan.toString() - assert(!planText.contains("__delta_internal_is_row_deleted")) - assert(!planText.contains("__delta_internal_row_index")) - } else { - assert(executedPlan.collect { case _: DeltaScanTransformer => true }.isEmpty) + if (SparkVersionUtil.gteSpark35) { + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.nonEmpty, planText) + assert(!planText.contains("__delta_internal_is_row_deleted")) + assert(!planText.contains("__delta_internal_row_index")) + } else { + assert(executedPlan.collect { case _: DeltaScanTransformer => true }.isEmpty, planText) + } + checkAnswer(df, df1) } - checkAnswer(df, df1) } } From cc6dbfd8b4318ed6d64b36d2c9546943e898d817 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 19:09:26 +0300 Subject: [PATCH 12/16] [VL][Delta] Broaden DV native scan assertion --- .../apache/gluten/execution/DeltaSuite.scala | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala b/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala index a9a3cc87a0a..f84b1315953 100644 --- a/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala +++ b/gluten-delta/src/test/scala/org/apache/gluten/execution/DeltaSuite.scala @@ -18,6 +18,7 @@ package org.apache.gluten.execution import org.apache.spark.SparkConf import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.types._ import org.apache.spark.util.SparkVersionUtil @@ -44,6 +45,15 @@ abstract class DeltaSuite extends WholeStageTransformerSuite { .set("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") } + private def hasNativeDeltaMorScan(plan: SparkPlan): Boolean = { + collect(plan) { + case _: DeltaScanTransformer => true + case scan: BatchScanExecTransformer + if scan.scan.getClass.getSimpleName == "ParquetScan" => + true + }.nonEmpty + } + // IdMapping is supported in Delta 2.2 (related to Spark3.3.1) test("column mapping mode = id") { withTable("delta_cm1") { @@ -213,16 +223,20 @@ abstract class DeltaSuite extends WholeStageTransformerSuite { withSQLConf("spark.databricks.delta.deletionVectors.useMetadataRowIndex" -> "true") { spark.sql(s"DELETE FROM delta.`$path` WHERE id IN (${values2.mkString(", ")})") val df = spark.read.format("delta").load(path) + checkAnswer(df, df1) val executedPlan = df.queryExecution.executedPlan val planText = executedPlan.toString() if (SparkVersionUtil.gteSpark35) { - assert(executedPlan.collect { case _: DeltaScanTransformer => true }.nonEmpty, planText) + assert(hasNativeDeltaMorScan(executedPlan), planText) + assert(!planText.contains("fallback Delta DV DML row-index scan"), planText) + assert( + !planText.contains("fallback Delta DV scan without metadata row index"), + planText) assert(!planText.contains("__delta_internal_is_row_deleted")) assert(!planText.contains("__delta_internal_row_index")) } else { - assert(executedPlan.collect { case _: DeltaScanTransformer => true }.isEmpty, planText) + assert(!hasNativeDeltaMorScan(executedPlan), planText) } - checkAnswer(df, df1) } } } From 20e45dc81c54dd4ffdb16c057029e4057623a9e0 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 20:19:29 +0300 Subject: [PATCH 13/16] [VL][Delta] Tighten DELETE DV metric assertions --- .../spark/sql/delta/DeleteSQLSuite.scala | 23 +++++++++++++------ .../spark/sql/delta/DeleteSQLSuite.scala | 23 +++++++++++++------ 2 files changed, 32 insertions(+), 14 deletions(-) diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala index 9a7317df8af..87707f1fa1f 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala @@ -179,6 +179,19 @@ class DeleteSQLWithDeletionVectorsSuite } } + def assertDeleteMetricAtLeast(key: String, expected: Long): Unit = { + val metrics = io.delta.tables.DeltaTable + .forPath(path) + .history() + .select("operationMetrics") + .take(1) + .head + .getMap(0) + .asInstanceOf[Map[String, String]] + .map { case (metricKey, value) => metricKey -> value.toLong } + assert(metrics.getOrElse(key, -1L) >= expected, s"Unexpected metric $key: $metrics") + } + executeDelete(s"delta.`$path`", "id % 3 = 0") assertRows(1, 2, 4, 5, 7, 8) assertActiveDeletionVectors(expectedFiles = 1, expectedCardinality = 4) @@ -193,19 +206,15 @@ class DeleteSQLWithDeletionVectorsSuite assertActiveDeletionVectors(expectedFiles = 1, expectedCardinality = 7) assertDeleteMetrics( "numDeletedRows" -> 3L, - "numDeletionVectorsAdded" -> 0L, - "numDeletionVectorsUpdated" -> 1L, - "numDeletionVectorsRemoved" -> 0L) + "numDeletionVectorsUpdated" -> 1L) executeDelete(s"delta.`$path`", "id IN (1, 2, 8)") assertRows() assertActiveDeletionVectors(expectedFiles = 0, expectedCardinality = 0) assertDeleteMetrics( "numDeletedRows" -> 3L, - "numRemovedFiles" -> 1L, - "numDeletionVectorsAdded" -> 0L, - "numDeletionVectorsUpdated" -> 0L, - "numDeletionVectorsRemoved" -> 1L) + "numRemovedFiles" -> 1L) + assertDeleteMetricAtLeast("numDeletionVectorsRemoved", 1L) } } } diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala index 275f3b6c6f4..666c803de14 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeleteSQLSuite.scala @@ -178,6 +178,19 @@ class DeleteSQLWithDeletionVectorsSuite extends DeleteSQLSuite } } + def assertDeleteMetricAtLeast(key: String, expected: Long): Unit = { + val metrics = io.delta.tables.DeltaTable + .forPath(path) + .history() + .select("operationMetrics") + .take(1) + .head + .getMap(0) + .asInstanceOf[Map[String, String]] + .map { case (metricKey, value) => metricKey -> value.toLong } + assert(metrics.getOrElse(key, -1L) >= expected, s"Unexpected metric $key: $metrics") + } + executeDelete(s"delta.`$path`", "id % 3 = 0") assertRows(1, 2, 4, 5, 7, 8) assertActiveDeletionVectors(expectedFiles = 1, expectedCardinality = 4) @@ -192,19 +205,15 @@ class DeleteSQLWithDeletionVectorsSuite extends DeleteSQLSuite assertActiveDeletionVectors(expectedFiles = 1, expectedCardinality = 7) assertDeleteMetrics( "numDeletedRows" -> 3L, - "numDeletionVectorsAdded" -> 0L, - "numDeletionVectorsUpdated" -> 1L, - "numDeletionVectorsRemoved" -> 0L) + "numDeletionVectorsUpdated" -> 1L) executeDelete(s"delta.`$path`", "id IN (1, 2, 8)") assertRows() assertActiveDeletionVectors(expectedFiles = 0, expectedCardinality = 0) assertDeleteMetrics( "numDeletedRows" -> 3L, - "numRemovedFiles" -> 1L, - "numDeletionVectorsAdded" -> 0L, - "numDeletionVectorsUpdated" -> 0L, - "numDeletionVectorsRemoved" -> 1L) + "numRemovedFiles" -> 1L) + assertDeleteMetricAtLeast("numDeletionVectorsRemoved", 1L) } } } From 4fcecde423cfdf5171d8b12c4f4778a929961689 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 15:05:04 +0300 Subject: [PATCH 14/16] [VL][Delta] Add DELETE DV diagnostics benchmark --- .../DeltaDeleteDeletionVectorBenchmark.scala | 320 ++++++++++++++++++ .../DeltaDeleteDeletionVectorBenchmark.scala | 320 ++++++++++++++++++ 2 files changed, 640 insertions(+) create mode 100644 backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala create mode 100644 backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala new file mode 100644 index 00000000000..44dbee09a7d --- /dev/null +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala @@ -0,0 +1,320 @@ +/* + * 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 + +import org.apache.gluten.config.{GlutenConfig, VeloxDeltaConfig} + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.util.Utils + +import io.delta.sql.DeltaSparkSessionExtension + +import java.io.File +import java.nio.charset.StandardCharsets +import java.util.Locale + +import scala.concurrent.duration.Duration +import scala.util.Try + +/** + * Focused benchmark for Delta DELETE with persistent deletion vectors. + * + * Usage: + * {{{ + * org.apache.spark.sql.delta.DeltaDeleteDeletionVectorBenchmark [rows] [files] [iterations] [mode] + * }}} + * + * Modes: + * - create: DELETE creates deletion vectors on a fresh table + * - update: DELETE updates existing deletion vectors + * - all: run both modes + */ +object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { + private val EnableNativeDmlRowIndexScan = + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" + + private case class BenchmarkConf( + rowCount: Long = 1000 * 1000, + files: Int = 8, + iterations: Int = 3, + mode: String = "all") + + private case class DeleteConfs( + glutenEnabled: Boolean, + nativeWriteEnabled: Boolean, + nativeDmlRowIndexScanEnabled: Boolean) + + private case class DeleteResult( + activeFiles: Long, + filesWithDvs: Long, + dvCardinality: Long, + dvPayloadBytes: Long) + + private var sparkSession: SparkSession = _ + private var benchmarkRoot: File = _ + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val conf = parseArgs(mainArgs) + sparkSession = createSparkSession(conf) + benchmarkRoot = Utils.createTempDir(namePrefix = "delta-delete-dv-benchmark") + + conf.mode match { + case "create" => + runDeleteBenchmark( + name = "Delta DELETE creates deletion vectors", + conf = conf, + existingDv = false, + measuredPredicate = "id % 10 = 0") + case "update" => + runDeleteBenchmark( + name = "Delta DELETE updates existing deletion vectors", + conf = conf, + existingDv = true, + measuredPredicate = "id % 10 = 1") + case "all" => + runDeleteBenchmark( + name = "Delta DELETE creates deletion vectors", + conf = conf, + existingDv = false, + measuredPredicate = "id % 10 = 0") + runDeleteBenchmark( + name = "Delta DELETE updates existing deletion vectors", + conf = conf, + existingDv = true, + measuredPredicate = "id % 10 = 1") + case other => + throw new IllegalArgumentException( + s"Unknown mode '$other'. Expected create, update, or all.") + } + } + + override def afterAll(): Unit = { + if (sparkSession != null) { + sparkSession.stop() + sparkSession = null + } + if (benchmarkRoot != null) { + Utils.deleteRecursively(benchmarkRoot) + benchmarkRoot = null + } + } + + private def spark: SparkSession = sparkSession + + private def parseArgs(args: Array[String]): BenchmarkConf = { + val defaults = BenchmarkConf() + BenchmarkConf( + rowCount = args.headOption.map(_.toLong).getOrElse(defaults.rowCount), + files = args.lift(1).map(_.toInt).getOrElse(defaults.files), + iterations = args.lift(2).map(_.toInt).getOrElse(defaults.iterations), + mode = args.lift(3).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.mode) + ) + } + + private def createSparkSession(conf: BenchmarkConf): SparkSession = { + val sparkConf = new SparkConf() + .setAppName("DeltaDeleteDeletionVectorBenchmark") + .setIfMissing("spark.master", "local[4]") + .set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, classOf[DeltaSparkSessionExtension].getName) + .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.default.parallelism", conf.files.toString) + .set("spark.sql.shuffle.partitions", conf.files.toString) + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "4g") + .set(SQLConf.ANSI_ENABLED.key, "false") + .set(GlutenConfig.GLUTEN_ANSI_FALLBACK_ENABLED.key, "false") + .set(GlutenConfig.FALLBACK_REPORTER_ENABLED.key, "false") + .set(VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key, "true") + .set(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key, "true") + .set( + DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey, + "true") + .set(DeltaSQLConf.DELTA_COLLECT_STATS.key, "false") + + SparkSession.builder.config(sparkConf).getOrCreate() + } + + private def runDeleteBenchmark( + name: String, + conf: BenchmarkConf, + existingDv: Boolean, + measuredPredicate: String): Unit = { + val sparkPaths = prepareTables(s"$name-spark", conf, existingDv) + val glutenPaths = prepareTables(s"$name-gluten", conf, existingDv) + val benchmark = new Benchmark( + name = s"$name (${conf.rowCount} rows, ${conf.files} files)", + valuesPerIteration = conf.rowCount, + minNumIters = 1, + warmupTime = Duration.Zero, + minTime = Duration.Zero, + outputPerIteration = true, + output = output + ) + + benchmark.addCase("Spark DELETE DV (Gluten disabled)", conf.iterations) { + iteration => + val result = runDelete( + sparkPaths(iteration), + measuredPredicate, + DeleteConfs( + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false)) + validateDeleteResult(result, existingDv) + printFirstIterationResult(iteration, "spark", result) + } + + benchmark.addCase("Gluten DELETE DV (native write + DML row-index scan)", conf.iterations) { + iteration => + val result = runDelete( + glutenPaths(iteration), + measuredPredicate, + DeleteConfs( + glutenEnabled = true, + nativeWriteEnabled = true, + nativeDmlRowIndexScanEnabled = true)) + validateDeleteResult(result, existingDv) + printFirstIterationResult(iteration, "gluten-native", result) + } + + benchmark.run() + } + + private def prepareTables( + prefix: String, + conf: BenchmarkConf, + existingDv: Boolean): IndexedSeq[String] = { + (0 until conf.iterations).map { + iteration => + val path = new File( + benchmarkRoot, + s"${sanitize(prefix)}-$iteration").getCanonicalPath + writeTable(path, conf) + if (existingDv) { + val result = runDelete( + path, + "id % 10 = 9", + DeleteConfs( + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false)) + validateDeleteResult(result, existingDv = false) + } + path + } + } + + private def writeTable(path: String, conf: BenchmarkConf): Unit = { + spark + .range(conf.rowCount) + .repartition(conf.files) + .selectExpr( + "id", + s"cast(id % ${math.max(conf.files, 1)} as int) as part", + "cast(id % 1000 as int) as payload") + .write + .format("delta") + .option(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.key, "true") + .mode("overwrite") + .save(path) + } + + private def runDelete( + path: String, + predicate: String, + confs: DeleteConfs): DeleteResult = { + withConfs( + "spark.gluten.enabled" -> confs.glutenEnabled.toString, + VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key -> confs.nativeWriteEnabled.toString, + EnableNativeDmlRowIndexScan -> confs.nativeDmlRowIndexScanEnabled.toString, + DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true", + DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey -> "true" + ) { + spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + } + collectDeleteResult(path) + } + + private def collectDeleteResult(path: String): DeleteResult = { + val files = DeltaLog.forTable(spark, path).update().allFiles.collect() + val filesWithDvs = files.filter(_.deletionVector != null) + DeleteResult( + activeFiles = files.length, + filesWithDvs = filesWithDvs.length, + dvCardinality = filesWithDvs.map(_.deletionVector.cardinality).sum, + dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum + ) + } + + private def validateDeleteResult(result: DeleteResult, existingDv: Boolean): Unit = { + require(result.filesWithDvs > 0, s"Expected deletion vectors, got $result") + require(result.dvCardinality > 0, s"Expected deleted-row cardinality, got $result") + if (existingDv) { + require( + result.dvCardinality > result.filesWithDvs, + s"Expected existing-DV update to retain non-trivial cardinality, got $result") + } + } + + private def printFirstIterationResult( + iteration: Int, + label: String, + result: DeleteResult): Unit = { + if (iteration == 0) { + writeOutputLine( + s"$label result: activeFiles=${result.activeFiles}, " + + s"filesWithDvs=${result.filesWithDvs}, " + + s"dvCardinality=${result.dvCardinality}, " + + s"dvPayloadBytes=${result.dvPayloadBytes}") + } + } + + private def writeOutputLine(line: String): Unit = { + output match { + case Some(out) => + out.write((line + System.lineSeparator()).getBytes(StandardCharsets.UTF_8)) + out.flush() + case None => + println(line) + } + } + + private def withConfs[T](confs: (String, String)*)(f: => T): T = { + val previous = confs.map { + case (key, _) => key -> Try(spark.conf.get(key)).toOption + } + try { + confs.foreach { case (key, value) => spark.conf.set(key, value) } + f + } finally { + previous.foreach { + case (key, Some(value)) => spark.conf.set(key, value) + case (key, None) => spark.conf.unset(key) + } + } + } + + private def sanitize(name: String): String = + name.toLowerCase(Locale.ROOT).replaceAll("[^a-z0-9]+", "-") +} diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala new file mode 100644 index 00000000000..44dbee09a7d --- /dev/null +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala @@ -0,0 +1,320 @@ +/* + * 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 + +import org.apache.gluten.config.{GlutenConfig, VeloxDeltaConfig} + +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.delta.catalog.DeltaCatalog +import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} +import org.apache.spark.util.Utils + +import io.delta.sql.DeltaSparkSessionExtension + +import java.io.File +import java.nio.charset.StandardCharsets +import java.util.Locale + +import scala.concurrent.duration.Duration +import scala.util.Try + +/** + * Focused benchmark for Delta DELETE with persistent deletion vectors. + * + * Usage: + * {{{ + * org.apache.spark.sql.delta.DeltaDeleteDeletionVectorBenchmark [rows] [files] [iterations] [mode] + * }}} + * + * Modes: + * - create: DELETE creates deletion vectors on a fresh table + * - update: DELETE updates existing deletion vectors + * - all: run both modes + */ +object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { + private val EnableNativeDmlRowIndexScan = + "spark.gluten.sql.delta.enableNativeDmlRowIndexScan" + + private case class BenchmarkConf( + rowCount: Long = 1000 * 1000, + files: Int = 8, + iterations: Int = 3, + mode: String = "all") + + private case class DeleteConfs( + glutenEnabled: Boolean, + nativeWriteEnabled: Boolean, + nativeDmlRowIndexScanEnabled: Boolean) + + private case class DeleteResult( + activeFiles: Long, + filesWithDvs: Long, + dvCardinality: Long, + dvPayloadBytes: Long) + + private var sparkSession: SparkSession = _ + private var benchmarkRoot: File = _ + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val conf = parseArgs(mainArgs) + sparkSession = createSparkSession(conf) + benchmarkRoot = Utils.createTempDir(namePrefix = "delta-delete-dv-benchmark") + + conf.mode match { + case "create" => + runDeleteBenchmark( + name = "Delta DELETE creates deletion vectors", + conf = conf, + existingDv = false, + measuredPredicate = "id % 10 = 0") + case "update" => + runDeleteBenchmark( + name = "Delta DELETE updates existing deletion vectors", + conf = conf, + existingDv = true, + measuredPredicate = "id % 10 = 1") + case "all" => + runDeleteBenchmark( + name = "Delta DELETE creates deletion vectors", + conf = conf, + existingDv = false, + measuredPredicate = "id % 10 = 0") + runDeleteBenchmark( + name = "Delta DELETE updates existing deletion vectors", + conf = conf, + existingDv = true, + measuredPredicate = "id % 10 = 1") + case other => + throw new IllegalArgumentException( + s"Unknown mode '$other'. Expected create, update, or all.") + } + } + + override def afterAll(): Unit = { + if (sparkSession != null) { + sparkSession.stop() + sparkSession = null + } + if (benchmarkRoot != null) { + Utils.deleteRecursively(benchmarkRoot) + benchmarkRoot = null + } + } + + private def spark: SparkSession = sparkSession + + private def parseArgs(args: Array[String]): BenchmarkConf = { + val defaults = BenchmarkConf() + BenchmarkConf( + rowCount = args.headOption.map(_.toLong).getOrElse(defaults.rowCount), + files = args.lift(1).map(_.toInt).getOrElse(defaults.files), + iterations = args.lift(2).map(_.toInt).getOrElse(defaults.iterations), + mode = args.lift(3).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.mode) + ) + } + + private def createSparkSession(conf: BenchmarkConf): SparkSession = { + val sparkConf = new SparkConf() + .setAppName("DeltaDeleteDeletionVectorBenchmark") + .setIfMissing("spark.master", "local[4]") + .set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, classOf[DeltaSparkSessionExtension].getName) + .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.default.parallelism", conf.files.toString) + .set("spark.sql.shuffle.partitions", conf.files.toString) + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "4g") + .set(SQLConf.ANSI_ENABLED.key, "false") + .set(GlutenConfig.GLUTEN_ANSI_FALLBACK_ENABLED.key, "false") + .set(GlutenConfig.FALLBACK_REPORTER_ENABLED.key, "false") + .set(VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key, "true") + .set(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key, "true") + .set( + DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey, + "true") + .set(DeltaSQLConf.DELTA_COLLECT_STATS.key, "false") + + SparkSession.builder.config(sparkConf).getOrCreate() + } + + private def runDeleteBenchmark( + name: String, + conf: BenchmarkConf, + existingDv: Boolean, + measuredPredicate: String): Unit = { + val sparkPaths = prepareTables(s"$name-spark", conf, existingDv) + val glutenPaths = prepareTables(s"$name-gluten", conf, existingDv) + val benchmark = new Benchmark( + name = s"$name (${conf.rowCount} rows, ${conf.files} files)", + valuesPerIteration = conf.rowCount, + minNumIters = 1, + warmupTime = Duration.Zero, + minTime = Duration.Zero, + outputPerIteration = true, + output = output + ) + + benchmark.addCase("Spark DELETE DV (Gluten disabled)", conf.iterations) { + iteration => + val result = runDelete( + sparkPaths(iteration), + measuredPredicate, + DeleteConfs( + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false)) + validateDeleteResult(result, existingDv) + printFirstIterationResult(iteration, "spark", result) + } + + benchmark.addCase("Gluten DELETE DV (native write + DML row-index scan)", conf.iterations) { + iteration => + val result = runDelete( + glutenPaths(iteration), + measuredPredicate, + DeleteConfs( + glutenEnabled = true, + nativeWriteEnabled = true, + nativeDmlRowIndexScanEnabled = true)) + validateDeleteResult(result, existingDv) + printFirstIterationResult(iteration, "gluten-native", result) + } + + benchmark.run() + } + + private def prepareTables( + prefix: String, + conf: BenchmarkConf, + existingDv: Boolean): IndexedSeq[String] = { + (0 until conf.iterations).map { + iteration => + val path = new File( + benchmarkRoot, + s"${sanitize(prefix)}-$iteration").getCanonicalPath + writeTable(path, conf) + if (existingDv) { + val result = runDelete( + path, + "id % 10 = 9", + DeleteConfs( + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false)) + validateDeleteResult(result, existingDv = false) + } + path + } + } + + private def writeTable(path: String, conf: BenchmarkConf): Unit = { + spark + .range(conf.rowCount) + .repartition(conf.files) + .selectExpr( + "id", + s"cast(id % ${math.max(conf.files, 1)} as int) as part", + "cast(id % 1000 as int) as payload") + .write + .format("delta") + .option(DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.key, "true") + .mode("overwrite") + .save(path) + } + + private def runDelete( + path: String, + predicate: String, + confs: DeleteConfs): DeleteResult = { + withConfs( + "spark.gluten.enabled" -> confs.glutenEnabled.toString, + VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key -> confs.nativeWriteEnabled.toString, + EnableNativeDmlRowIndexScan -> confs.nativeDmlRowIndexScanEnabled.toString, + DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true", + DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey -> "true" + ) { + spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + } + collectDeleteResult(path) + } + + private def collectDeleteResult(path: String): DeleteResult = { + val files = DeltaLog.forTable(spark, path).update().allFiles.collect() + val filesWithDvs = files.filter(_.deletionVector != null) + DeleteResult( + activeFiles = files.length, + filesWithDvs = filesWithDvs.length, + dvCardinality = filesWithDvs.map(_.deletionVector.cardinality).sum, + dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum + ) + } + + private def validateDeleteResult(result: DeleteResult, existingDv: Boolean): Unit = { + require(result.filesWithDvs > 0, s"Expected deletion vectors, got $result") + require(result.dvCardinality > 0, s"Expected deleted-row cardinality, got $result") + if (existingDv) { + require( + result.dvCardinality > result.filesWithDvs, + s"Expected existing-DV update to retain non-trivial cardinality, got $result") + } + } + + private def printFirstIterationResult( + iteration: Int, + label: String, + result: DeleteResult): Unit = { + if (iteration == 0) { + writeOutputLine( + s"$label result: activeFiles=${result.activeFiles}, " + + s"filesWithDvs=${result.filesWithDvs}, " + + s"dvCardinality=${result.dvCardinality}, " + + s"dvPayloadBytes=${result.dvPayloadBytes}") + } + } + + private def writeOutputLine(line: String): Unit = { + output match { + case Some(out) => + out.write((line + System.lineSeparator()).getBytes(StandardCharsets.UTF_8)) + out.flush() + case None => + println(line) + } + } + + private def withConfs[T](confs: (String, String)*)(f: => T): T = { + val previous = confs.map { + case (key, _) => key -> Try(spark.conf.get(key)).toOption + } + try { + confs.foreach { case (key, value) => spark.conf.set(key, value) } + f + } finally { + previous.foreach { + case (key, Some(value)) => spark.conf.set(key, value) + case (key, None) => spark.conf.unset(key) + } + } + } + + private def sanitize(name: String): String = + name.toLowerCase(Locale.ROOT).replaceAll("[^a-z0-9]+", "-") +} From be55fb416d5a2015165bee3281460ce944221992 Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 17:11:50 +0300 Subject: [PATCH 15/16] [VL][Delta] Add Spark mode for DELETE DV benchmark --- .../DeltaDeleteDeletionVectorBenchmark.scala | 229 ++++++++++++------ .../DeltaDeleteDeletionVectorBenchmark.scala | 229 ++++++++++++------ 2 files changed, 302 insertions(+), 156 deletions(-) diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala index 44dbee09a7d..fec83846587 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala @@ -40,13 +40,11 @@ import scala.util.Try * * Usage: * {{{ - * org.apache.spark.sql.delta.DeltaDeleteDeletionVectorBenchmark [rows] [files] [iterations] [mode] + * org.apache.spark.sql.delta.DeltaDeleteDeletionVectorBenchmark \ + * [rows] [files] [iterations] [deleteMode] [executionMode] * }}} * - * Modes: - * - create: DELETE creates deletion vectors on a fresh table - * - update: DELETE updates existing deletion vectors - * - all: run both modes + * Delete modes: create, update, all. Execution modes: spark, gluten, all. */ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { private val EnableNativeDmlRowIndexScan = @@ -56,7 +54,13 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { rowCount: Long = 1000 * 1000, files: Int = 8, iterations: Int = 3, - mode: String = "all") + deleteMode: String = "all", + executionMode: String = "spark") + + private case class ExecutionMode( + label: String, + withGlutenPlugin: Boolean, + deleteConfs: DeleteConfs) private case class DeleteConfs( glutenEnabled: Boolean, @@ -67,51 +71,70 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { activeFiles: Long, filesWithDvs: Long, dvCardinality: Long, - dvPayloadBytes: Long) + dvPayloadBytes: Long, + finalRows: Long) private var sparkSession: SparkSession = _ private var benchmarkRoot: File = _ override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val conf = parseArgs(mainArgs) - sparkSession = createSparkSession(conf) - benchmarkRoot = Utils.createTempDir(namePrefix = "delta-delete-dv-benchmark") - - conf.mode match { - case "create" => - runDeleteBenchmark( - name = "Delta DELETE creates deletion vectors", - conf = conf, - existingDv = false, - measuredPredicate = "id % 10 = 0") - case "update" => - runDeleteBenchmark( - name = "Delta DELETE updates existing deletion vectors", - conf = conf, - existingDv = true, - measuredPredicate = "id % 10 = 1") - case "all" => - runDeleteBenchmark( - name = "Delta DELETE creates deletion vectors", - conf = conf, - existingDv = false, - measuredPredicate = "id % 10 = 0") - runDeleteBenchmark( - name = "Delta DELETE updates existing deletion vectors", - conf = conf, - existingDv = true, - measuredPredicate = "id % 10 = 1") - case other => - throw new IllegalArgumentException( - s"Unknown mode '$other'. Expected create, update, or all.") + executionModes(conf.executionMode).foreach { + mode => + sparkSession = createSparkSession(conf, mode) + benchmarkRoot = Utils.createTempDir( + namePrefix = s"delta-delete-dv-benchmark-${mode.label}") + try { + conf.deleteMode match { + case "create" => + runDeleteBenchmark( + name = "Delta DELETE creates deletion vectors", + conf = conf, + mode = mode, + existingDv = false, + measuredPredicate = "id % 10 = 0", + expectedDeletedMods = Seq(0)) + case "update" => + runDeleteBenchmark( + name = "Delta DELETE updates existing deletion vectors", + conf = conf, + mode = mode, + existingDv = true, + measuredPredicate = "id % 10 = 1", + expectedDeletedMods = Seq(9, 1) + ) + case "all" => + runDeleteBenchmark( + name = "Delta DELETE creates deletion vectors", + conf = conf, + mode = mode, + existingDv = false, + measuredPredicate = "id % 10 = 0", + expectedDeletedMods = Seq(0)) + runDeleteBenchmark( + name = "Delta DELETE updates existing deletion vectors", + conf = conf, + mode = mode, + existingDv = true, + measuredPredicate = "id % 10 = 1", + expectedDeletedMods = Seq(9, 1) + ) + case other => + throw new IllegalArgumentException( + s"Unknown delete mode '$other'. Expected create, update, or all.") + } + } finally { + stopSpark() + if (benchmarkRoot != null) { + Utils.deleteRecursively(benchmarkRoot) + benchmarkRoot = null + } + } } } override def afterAll(): Unit = { - if (sparkSession != null) { - sparkSession.stop() - sparkSession = null - } + stopSpark() if (benchmarkRoot != null) { Utils.deleteRecursively(benchmarkRoot) benchmarkRoot = null @@ -126,44 +149,79 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { rowCount = args.headOption.map(_.toLong).getOrElse(defaults.rowCount), files = args.lift(1).map(_.toInt).getOrElse(defaults.files), iterations = args.lift(2).map(_.toInt).getOrElse(defaults.iterations), - mode = args.lift(3).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.mode) + deleteMode = args.lift(3).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.deleteMode), + executionMode = + args.lift(4).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.executionMode) ) } - private def createSparkSession(conf: BenchmarkConf): SparkSession = { + private def executionModes(mode: String): Seq[ExecutionMode] = { + val sparkOnly = ExecutionMode( + label = "spark", + withGlutenPlugin = false, + deleteConfs = DeleteConfs( + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false)) + val glutenNative = ExecutionMode( + label = "gluten-native", + withGlutenPlugin = true, + deleteConfs = DeleteConfs( + glutenEnabled = true, + nativeWriteEnabled = true, + nativeDmlRowIndexScanEnabled = true)) + mode match { + case "spark" => Seq(sparkOnly) + case "gluten" => Seq(glutenNative) + case "all" => Seq(sparkOnly, glutenNative) + case other => + throw new IllegalArgumentException( + s"Unknown execution mode '$other'. Expected spark, gluten, or all.") + } + } + + private def createSparkSession(conf: BenchmarkConf, mode: ExecutionMode): SparkSession = { val sparkConf = new SparkConf() - .setAppName("DeltaDeleteDeletionVectorBenchmark") + .setAppName(s"DeltaDeleteDeletionVectorBenchmark-${mode.label}") .setIfMissing("spark.master", "local[4]") .set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, classOf[DeltaSparkSessionExtension].getName) .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) - .set("spark.plugins", "org.apache.gluten.GlutenPlugin") - .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.default.parallelism", conf.files.toString) .set("spark.sql.shuffle.partitions", conf.files.toString) - .set("spark.memory.offHeap.enabled", "true") - .set("spark.memory.offHeap.size", "4g") .set(SQLConf.ANSI_ENABLED.key, "false") .set(GlutenConfig.GLUTEN_ANSI_FALLBACK_ENABLED.key, "false") .set(GlutenConfig.FALLBACK_REPORTER_ENABLED.key, "false") - .set(VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key, "true") + .set("spark.gluten.enabled", mode.deleteConfs.glutenEnabled.toString) + .set(VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key, mode.deleteConfs.nativeWriteEnabled.toString) + .set(EnableNativeDmlRowIndexScan, mode.deleteConfs.nativeDmlRowIndexScanEnabled.toString) .set(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key, "true") .set( DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey, "true") .set(DeltaSQLConf.DELTA_COLLECT_STATS.key, "false") + if (mode.withGlutenPlugin) { + sparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "4g") + } + SparkSession.builder.config(sparkConf).getOrCreate() } private def runDeleteBenchmark( name: String, conf: BenchmarkConf, + mode: ExecutionMode, existingDv: Boolean, - measuredPredicate: String): Unit = { - val sparkPaths = prepareTables(s"$name-spark", conf, existingDv) - val glutenPaths = prepareTables(s"$name-gluten", conf, existingDv) + measuredPredicate: String, + expectedDeletedMods: Seq[Int]): Unit = { + val paths = prepareTables(s"$name-${mode.label}", conf, existingDv) + val expectedFinalRows = expectedRemainingRows(conf.rowCount, expectedDeletedMods) val benchmark = new Benchmark( - name = s"$name (${conf.rowCount} rows, ${conf.files} files)", + name = s"$name ${mode.label} (${conf.rowCount} rows, ${conf.files} files)", valuesPerIteration = conf.rowCount, minNumIters = 1, warmupTime = Duration.Zero, @@ -172,30 +230,14 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { output = output ) - benchmark.addCase("Spark DELETE DV (Gluten disabled)", conf.iterations) { + benchmark.addCase(s"${mode.label} DELETE DV", conf.iterations) { iteration => val result = runDelete( - sparkPaths(iteration), + paths(iteration), measuredPredicate, - DeleteConfs( - glutenEnabled = false, - nativeWriteEnabled = false, - nativeDmlRowIndexScanEnabled = false)) - validateDeleteResult(result, existingDv) - printFirstIterationResult(iteration, "spark", result) - } - - benchmark.addCase("Gluten DELETE DV (native write + DML row-index scan)", conf.iterations) { - iteration => - val result = runDelete( - glutenPaths(iteration), - measuredPredicate, - DeleteConfs( - glutenEnabled = true, - nativeWriteEnabled = true, - nativeDmlRowIndexScanEnabled = true)) - validateDeleteResult(result, existingDv) - printFirstIterationResult(iteration, "gluten-native", result) + mode.deleteConfs) + validateDeleteResult(result, existingDv, expectedFinalRows) + printFirstIterationResult(iteration, mode.label, result) } benchmark.run() @@ -219,7 +261,10 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { glutenEnabled = false, nativeWriteEnabled = false, nativeDmlRowIndexScanEnabled = false)) - validateDeleteResult(result, existingDv = false) + validateDeleteResult( + result, + existingDv = false, + expectedFinalRows = expectedRemainingRows(conf.rowCount, Seq(9))) } path } @@ -259,17 +304,25 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { private def collectDeleteResult(path: String): DeleteResult = { val files = DeltaLog.forTable(spark, path).update().allFiles.collect() val filesWithDvs = files.filter(_.deletionVector != null) + val finalRows = spark.read.format("delta").load(path).count() DeleteResult( activeFiles = files.length, filesWithDvs = filesWithDvs.length, dvCardinality = filesWithDvs.map(_.deletionVector.cardinality).sum, - dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum + dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum, + finalRows = finalRows ) } - private def validateDeleteResult(result: DeleteResult, existingDv: Boolean): Unit = { + private def validateDeleteResult( + result: DeleteResult, + existingDv: Boolean, + expectedFinalRows: Long): Unit = { require(result.filesWithDvs > 0, s"Expected deletion vectors, got $result") require(result.dvCardinality > 0, s"Expected deleted-row cardinality, got $result") + require( + result.finalRows == expectedFinalRows, + s"Expected $expectedFinalRows final rows, got $result") if (existingDv) { require( result.dvCardinality > result.filesWithDvs, @@ -286,7 +339,8 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { s"$label result: activeFiles=${result.activeFiles}, " + s"filesWithDvs=${result.filesWithDvs}, " + s"dvCardinality=${result.dvCardinality}, " + - s"dvPayloadBytes=${result.dvPayloadBytes}") + s"dvPayloadBytes=${result.dvPayloadBytes}, " + + s"finalRows=${result.finalRows}") } } @@ -315,6 +369,25 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { } } + private def expectedRemainingRows(rowCount: Long, deletedMods: Seq[Int]): Long = + rowCount - deletedMods.distinct.map(countRowsWithMod(rowCount, _)).sum + + private def countRowsWithMod(rowCount: Long, mod: Int): Long = { + require(mod >= 0 && mod < 10, s"Expected modulo in [0, 10), got $mod") + if (rowCount <= mod) { + 0L + } else { + ((rowCount - 1 - mod) / 10) + 1 + } + } + + private def stopSpark(): Unit = { + if (sparkSession != null) { + sparkSession.stop() + sparkSession = null + } + } + private def sanitize(name: String): String = name.toLowerCase(Locale.ROOT).replaceAll("[^a-z0-9]+", "-") } diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala index 44dbee09a7d..fec83846587 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala @@ -40,13 +40,11 @@ import scala.util.Try * * Usage: * {{{ - * org.apache.spark.sql.delta.DeltaDeleteDeletionVectorBenchmark [rows] [files] [iterations] [mode] + * org.apache.spark.sql.delta.DeltaDeleteDeletionVectorBenchmark \ + * [rows] [files] [iterations] [deleteMode] [executionMode] * }}} * - * Modes: - * - create: DELETE creates deletion vectors on a fresh table - * - update: DELETE updates existing deletion vectors - * - all: run both modes + * Delete modes: create, update, all. Execution modes: spark, gluten, all. */ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { private val EnableNativeDmlRowIndexScan = @@ -56,7 +54,13 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { rowCount: Long = 1000 * 1000, files: Int = 8, iterations: Int = 3, - mode: String = "all") + deleteMode: String = "all", + executionMode: String = "spark") + + private case class ExecutionMode( + label: String, + withGlutenPlugin: Boolean, + deleteConfs: DeleteConfs) private case class DeleteConfs( glutenEnabled: Boolean, @@ -67,51 +71,70 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { activeFiles: Long, filesWithDvs: Long, dvCardinality: Long, - dvPayloadBytes: Long) + dvPayloadBytes: Long, + finalRows: Long) private var sparkSession: SparkSession = _ private var benchmarkRoot: File = _ override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val conf = parseArgs(mainArgs) - sparkSession = createSparkSession(conf) - benchmarkRoot = Utils.createTempDir(namePrefix = "delta-delete-dv-benchmark") - - conf.mode match { - case "create" => - runDeleteBenchmark( - name = "Delta DELETE creates deletion vectors", - conf = conf, - existingDv = false, - measuredPredicate = "id % 10 = 0") - case "update" => - runDeleteBenchmark( - name = "Delta DELETE updates existing deletion vectors", - conf = conf, - existingDv = true, - measuredPredicate = "id % 10 = 1") - case "all" => - runDeleteBenchmark( - name = "Delta DELETE creates deletion vectors", - conf = conf, - existingDv = false, - measuredPredicate = "id % 10 = 0") - runDeleteBenchmark( - name = "Delta DELETE updates existing deletion vectors", - conf = conf, - existingDv = true, - measuredPredicate = "id % 10 = 1") - case other => - throw new IllegalArgumentException( - s"Unknown mode '$other'. Expected create, update, or all.") + executionModes(conf.executionMode).foreach { + mode => + sparkSession = createSparkSession(conf, mode) + benchmarkRoot = Utils.createTempDir( + namePrefix = s"delta-delete-dv-benchmark-${mode.label}") + try { + conf.deleteMode match { + case "create" => + runDeleteBenchmark( + name = "Delta DELETE creates deletion vectors", + conf = conf, + mode = mode, + existingDv = false, + measuredPredicate = "id % 10 = 0", + expectedDeletedMods = Seq(0)) + case "update" => + runDeleteBenchmark( + name = "Delta DELETE updates existing deletion vectors", + conf = conf, + mode = mode, + existingDv = true, + measuredPredicate = "id % 10 = 1", + expectedDeletedMods = Seq(9, 1) + ) + case "all" => + runDeleteBenchmark( + name = "Delta DELETE creates deletion vectors", + conf = conf, + mode = mode, + existingDv = false, + measuredPredicate = "id % 10 = 0", + expectedDeletedMods = Seq(0)) + runDeleteBenchmark( + name = "Delta DELETE updates existing deletion vectors", + conf = conf, + mode = mode, + existingDv = true, + measuredPredicate = "id % 10 = 1", + expectedDeletedMods = Seq(9, 1) + ) + case other => + throw new IllegalArgumentException( + s"Unknown delete mode '$other'. Expected create, update, or all.") + } + } finally { + stopSpark() + if (benchmarkRoot != null) { + Utils.deleteRecursively(benchmarkRoot) + benchmarkRoot = null + } + } } } override def afterAll(): Unit = { - if (sparkSession != null) { - sparkSession.stop() - sparkSession = null - } + stopSpark() if (benchmarkRoot != null) { Utils.deleteRecursively(benchmarkRoot) benchmarkRoot = null @@ -126,44 +149,79 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { rowCount = args.headOption.map(_.toLong).getOrElse(defaults.rowCount), files = args.lift(1).map(_.toInt).getOrElse(defaults.files), iterations = args.lift(2).map(_.toInt).getOrElse(defaults.iterations), - mode = args.lift(3).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.mode) + deleteMode = args.lift(3).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.deleteMode), + executionMode = + args.lift(4).map(_.toLowerCase(Locale.ROOT)).getOrElse(defaults.executionMode) ) } - private def createSparkSession(conf: BenchmarkConf): SparkSession = { + private def executionModes(mode: String): Seq[ExecutionMode] = { + val sparkOnly = ExecutionMode( + label = "spark", + withGlutenPlugin = false, + deleteConfs = DeleteConfs( + glutenEnabled = false, + nativeWriteEnabled = false, + nativeDmlRowIndexScanEnabled = false)) + val glutenNative = ExecutionMode( + label = "gluten-native", + withGlutenPlugin = true, + deleteConfs = DeleteConfs( + glutenEnabled = true, + nativeWriteEnabled = true, + nativeDmlRowIndexScanEnabled = true)) + mode match { + case "spark" => Seq(sparkOnly) + case "gluten" => Seq(glutenNative) + case "all" => Seq(sparkOnly, glutenNative) + case other => + throw new IllegalArgumentException( + s"Unknown execution mode '$other'. Expected spark, gluten, or all.") + } + } + + private def createSparkSession(conf: BenchmarkConf, mode: ExecutionMode): SparkSession = { val sparkConf = new SparkConf() - .setAppName("DeltaDeleteDeletionVectorBenchmark") + .setAppName(s"DeltaDeleteDeletionVectorBenchmark-${mode.label}") .setIfMissing("spark.master", "local[4]") .set(StaticSQLConf.SPARK_SESSION_EXTENSIONS.key, classOf[DeltaSparkSessionExtension].getName) .set(SQLConf.V2_SESSION_CATALOG_IMPLEMENTATION.key, classOf[DeltaCatalog].getName) - .set("spark.plugins", "org.apache.gluten.GlutenPlugin") - .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") .set("spark.default.parallelism", conf.files.toString) .set("spark.sql.shuffle.partitions", conf.files.toString) - .set("spark.memory.offHeap.enabled", "true") - .set("spark.memory.offHeap.size", "4g") .set(SQLConf.ANSI_ENABLED.key, "false") .set(GlutenConfig.GLUTEN_ANSI_FALLBACK_ENABLED.key, "false") .set(GlutenConfig.FALLBACK_REPORTER_ENABLED.key, "false") - .set(VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key, "true") + .set("spark.gluten.enabled", mode.deleteConfs.glutenEnabled.toString) + .set(VeloxDeltaConfig.ENABLE_NATIVE_WRITE.key, mode.deleteConfs.nativeWriteEnabled.toString) + .set(EnableNativeDmlRowIndexScan, mode.deleteConfs.nativeDmlRowIndexScanEnabled.toString) .set(DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key, "true") .set( DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey, "true") .set(DeltaSQLConf.DELTA_COLLECT_STATS.key, "false") + if (mode.withGlutenPlugin) { + sparkConf + .set("spark.plugins", "org.apache.gluten.GlutenPlugin") + .set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager") + .set("spark.memory.offHeap.enabled", "true") + .set("spark.memory.offHeap.size", "4g") + } + SparkSession.builder.config(sparkConf).getOrCreate() } private def runDeleteBenchmark( name: String, conf: BenchmarkConf, + mode: ExecutionMode, existingDv: Boolean, - measuredPredicate: String): Unit = { - val sparkPaths = prepareTables(s"$name-spark", conf, existingDv) - val glutenPaths = prepareTables(s"$name-gluten", conf, existingDv) + measuredPredicate: String, + expectedDeletedMods: Seq[Int]): Unit = { + val paths = prepareTables(s"$name-${mode.label}", conf, existingDv) + val expectedFinalRows = expectedRemainingRows(conf.rowCount, expectedDeletedMods) val benchmark = new Benchmark( - name = s"$name (${conf.rowCount} rows, ${conf.files} files)", + name = s"$name ${mode.label} (${conf.rowCount} rows, ${conf.files} files)", valuesPerIteration = conf.rowCount, minNumIters = 1, warmupTime = Duration.Zero, @@ -172,30 +230,14 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { output = output ) - benchmark.addCase("Spark DELETE DV (Gluten disabled)", conf.iterations) { + benchmark.addCase(s"${mode.label} DELETE DV", conf.iterations) { iteration => val result = runDelete( - sparkPaths(iteration), + paths(iteration), measuredPredicate, - DeleteConfs( - glutenEnabled = false, - nativeWriteEnabled = false, - nativeDmlRowIndexScanEnabled = false)) - validateDeleteResult(result, existingDv) - printFirstIterationResult(iteration, "spark", result) - } - - benchmark.addCase("Gluten DELETE DV (native write + DML row-index scan)", conf.iterations) { - iteration => - val result = runDelete( - glutenPaths(iteration), - measuredPredicate, - DeleteConfs( - glutenEnabled = true, - nativeWriteEnabled = true, - nativeDmlRowIndexScanEnabled = true)) - validateDeleteResult(result, existingDv) - printFirstIterationResult(iteration, "gluten-native", result) + mode.deleteConfs) + validateDeleteResult(result, existingDv, expectedFinalRows) + printFirstIterationResult(iteration, mode.label, result) } benchmark.run() @@ -219,7 +261,10 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { glutenEnabled = false, nativeWriteEnabled = false, nativeDmlRowIndexScanEnabled = false)) - validateDeleteResult(result, existingDv = false) + validateDeleteResult( + result, + existingDv = false, + expectedFinalRows = expectedRemainingRows(conf.rowCount, Seq(9))) } path } @@ -259,17 +304,25 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { private def collectDeleteResult(path: String): DeleteResult = { val files = DeltaLog.forTable(spark, path).update().allFiles.collect() val filesWithDvs = files.filter(_.deletionVector != null) + val finalRows = spark.read.format("delta").load(path).count() DeleteResult( activeFiles = files.length, filesWithDvs = filesWithDvs.length, dvCardinality = filesWithDvs.map(_.deletionVector.cardinality).sum, - dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum + dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum, + finalRows = finalRows ) } - private def validateDeleteResult(result: DeleteResult, existingDv: Boolean): Unit = { + private def validateDeleteResult( + result: DeleteResult, + existingDv: Boolean, + expectedFinalRows: Long): Unit = { require(result.filesWithDvs > 0, s"Expected deletion vectors, got $result") require(result.dvCardinality > 0, s"Expected deleted-row cardinality, got $result") + require( + result.finalRows == expectedFinalRows, + s"Expected $expectedFinalRows final rows, got $result") if (existingDv) { require( result.dvCardinality > result.filesWithDvs, @@ -286,7 +339,8 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { s"$label result: activeFiles=${result.activeFiles}, " + s"filesWithDvs=${result.filesWithDvs}, " + s"dvCardinality=${result.dvCardinality}, " + - s"dvPayloadBytes=${result.dvPayloadBytes}") + s"dvPayloadBytes=${result.dvPayloadBytes}, " + + s"finalRows=${result.finalRows}") } } @@ -315,6 +369,25 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { } } + private def expectedRemainingRows(rowCount: Long, deletedMods: Seq[Int]): Long = + rowCount - deletedMods.distinct.map(countRowsWithMod(rowCount, _)).sum + + private def countRowsWithMod(rowCount: Long, mod: Int): Long = { + require(mod >= 0 && mod < 10, s"Expected modulo in [0, 10), got $mod") + if (rowCount <= mod) { + 0L + } else { + ((rowCount - 1 - mod) / 10) + 1 + } + } + + private def stopSpark(): Unit = { + if (sparkSession != null) { + sparkSession.stop() + sparkSession = null + } + } + private def sanitize(name: String): String = name.toLowerCase(Locale.ROOT).replaceAll("[^a-z0-9]+", "-") } From 104a3564f4d7d70155c0ce185c124db828ef92ca Mon Sep 17 00:00:00 2001 From: malinjawi Date: Mon, 1 Jun 2026 18:09:14 +0300 Subject: [PATCH 16/16] [VL][Delta] Add DELETE DV benchmark diagnostics --- .../DeltaDeleteDeletionVectorBenchmark.scala | 107 ++++++++++++++++-- .../DeltaDeleteDeletionVectorBenchmark.scala | 107 ++++++++++++++++-- 2 files changed, 194 insertions(+), 20 deletions(-) diff --git a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala index fec83846587..afac57dd3c0 100644 --- a/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala +++ b/backends-velox/src-delta33/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala @@ -17,12 +17,16 @@ package org.apache.spark.sql.delta import org.apache.gluten.config.{GlutenConfig, VeloxDeltaConfig} +import org.apache.gluten.execution.DeltaScanTransformer +import org.apache.gluten.extension.DeltaDeletionVectorDmlUtils +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.catalog.DeltaCatalog import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.util.Utils @@ -72,7 +76,18 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { filesWithDvs: Long, dvCardinality: Long, dvPayloadBytes: Long, - finalRows: Long) + finalRows: Long, + finalIdSum: BigInt, + deleteMs: Long, + validationMs: Long, + planSummary: PlanSummary) + + private case class PlanSummary( + deletePlans: Int, + glutenDeleteCommands: Int, + deltaScanTransformers: Int, + dmlRowIndexFallbackScans: Int, + fallbackReasons: Seq[String]) private var sparkSession: SparkSession = _ private var benchmarkRoot: File = _ @@ -220,6 +235,7 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { expectedDeletedMods: Seq[Int]): Unit = { val paths = prepareTables(s"$name-${mode.label}", conf, existingDv) val expectedFinalRows = expectedRemainingRows(conf.rowCount, expectedDeletedMods) + val expectedFinalIdSum = expectedRemainingIdSum(conf.rowCount, expectedDeletedMods) val benchmark = new Benchmark( name = s"$name ${mode.label} (${conf.rowCount} rows, ${conf.files} files)", valuesPerIteration = conf.rowCount, @@ -236,7 +252,7 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { paths(iteration), measuredPredicate, mode.deleteConfs) - validateDeleteResult(result, existingDv, expectedFinalRows) + validateDeleteResult(result, existingDv, expectedFinalRows, expectedFinalIdSum) printFirstIterationResult(iteration, mode.label, result) } @@ -264,7 +280,9 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { validateDeleteResult( result, existingDv = false, - expectedFinalRows = expectedRemainingRows(conf.rowCount, Seq(9))) + expectedFinalRows = expectedRemainingRows(conf.rowCount, Seq(9)), + expectedFinalIdSum = expectedRemainingIdSum(conf.rowCount, Seq(9)) + ) } path } @@ -296,33 +314,76 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true", DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey -> "true" ) { - spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + val deleteStartNs = System.nanoTime() + val executedPlans = DeltaTestUtils.withAllPlansCaptured(spark) { + spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + }.map(_.executedPlan) + val deleteMs = elapsedMs(deleteStartNs) + collectDeleteResult(path, deleteMs, summarizePlans(executedPlans)) } - collectDeleteResult(path) } - private def collectDeleteResult(path: String): DeleteResult = { + private def collectDeleteResult( + path: String, + deleteMs: Long, + planSummary: PlanSummary): DeleteResult = { + val validationStartNs = System.nanoTime() val files = DeltaLog.forTable(spark, path).update().allFiles.collect() val filesWithDvs = files.filter(_.deletionVector != null) - val finalRows = spark.read.format("delta").load(path).count() + val finalStats = spark.read + .format("delta") + .load(path) + .selectExpr( + "count(*) as final_rows", + "coalesce(sum(cast(id as decimal(38,0))), cast(0 as decimal(38,0))) as final_id_sum") + .head() DeleteResult( activeFiles = files.length, filesWithDvs = filesWithDvs.length, dvCardinality = filesWithDvs.map(_.deletionVector.cardinality).sum, dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum, - finalRows = finalRows + finalRows = finalStats.getLong(0), + finalIdSum = BigInt(finalStats.getDecimal(1).toBigInteger), + deleteMs = deleteMs, + validationMs = elapsedMs(validationStartNs), + planSummary = planSummary + ) + } + + private def summarizePlans(executedPlans: Seq[SparkPlan]): PlanSummary = { + val planNodes = executedPlans.flatMap(_.collect { case node: SparkPlan => node }) + val fileScans = planNodes.collect { case scan: FileSourceScanExec => scan } + val fallbackReasons = + planNodes.flatMap(plan => FallbackTags.getOption(plan).map(_.reason())).distinct.sorted + val dmlFallbackScans = fileScans.count { + scan => + DeltaDeletionVectorDmlUtils.isDeletionVectorDmlRowIndexScan(scan) && + FallbackTags + .getOption(scan) + .exists(_.reason().contains("fallback Delta DV DML row-index scan")) + } + PlanSummary( + deletePlans = executedPlans.length, + glutenDeleteCommands = planNodes.count(_.nodeName.contains("GlutenDeleteCommand")), + deltaScanTransformers = planNodes.count(_.isInstanceOf[DeltaScanTransformer]), + dmlRowIndexFallbackScans = dmlFallbackScans, + fallbackReasons = fallbackReasons ) } private def validateDeleteResult( result: DeleteResult, existingDv: Boolean, - expectedFinalRows: Long): Unit = { + expectedFinalRows: Long, + expectedFinalIdSum: BigInt): Unit = { require(result.filesWithDvs > 0, s"Expected deletion vectors, got $result") require(result.dvCardinality > 0, s"Expected deleted-row cardinality, got $result") require( result.finalRows == expectedFinalRows, s"Expected $expectedFinalRows final rows, got $result") + require( + result.finalIdSum == expectedFinalIdSum, + s"Expected final id sum $expectedFinalIdSum, got $result") if (existingDv) { require( result.dvCardinality > result.filesWithDvs, @@ -340,7 +401,15 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { s"filesWithDvs=${result.filesWithDvs}, " + s"dvCardinality=${result.dvCardinality}, " + s"dvPayloadBytes=${result.dvPayloadBytes}, " + - s"finalRows=${result.finalRows}") + s"finalRows=${result.finalRows}, " + + s"finalIdSum=${result.finalIdSum}, " + + s"deleteMs=${result.deleteMs}, " + + s"validationMs=${result.validationMs}, " + + s"deletePlans=${result.planSummary.deletePlans}, " + + s"glutenDeleteCommands=${result.planSummary.glutenDeleteCommands}, " + + s"deltaScanTransformers=${result.planSummary.deltaScanTransformers}, " + + s"dmlRowIndexFallbackScans=${result.planSummary.dmlRowIndexFallbackScans}, " + + s"fallbackReasons=${result.planSummary.fallbackReasons.mkString("[", "; ", "]")}") } } @@ -372,6 +441,21 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { private def expectedRemainingRows(rowCount: Long, deletedMods: Seq[Int]): Long = rowCount - deletedMods.distinct.map(countRowsWithMod(rowCount, _)).sum + private def expectedRemainingIdSum(rowCount: Long, deletedMods: Seq[Int]): BigInt = + sumRange(rowCount) - deletedMods.distinct.map(sumRowsWithMod(rowCount, _)).sum + + private def sumRange(rowCount: Long): BigInt = + BigInt(rowCount) * BigInt(rowCount - 1L) / 2 + + private def sumRowsWithMod(rowCount: Long, mod: Int): BigInt = { + val count = countRowsWithMod(rowCount, mod) + if (count == 0L) { + BigInt(0) + } else { + BigInt(count) * (BigInt(2L * mod) + BigInt(10L) * BigInt(count - 1L)) / 2 + } + } + private def countRowsWithMod(rowCount: Long, mod: Int): Long = { require(mod >= 0 && mod < 10, s"Expected modulo in [0, 10), got $mod") if (rowCount <= mod) { @@ -390,4 +474,7 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { private def sanitize(name: String): String = name.toLowerCase(Locale.ROOT).replaceAll("[^a-z0-9]+", "-") + + private def elapsedMs(startNs: Long): Long = + (System.nanoTime() - startNs) / (1000L * 1000L) } diff --git a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala index fec83846587..afac57dd3c0 100644 --- a/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala +++ b/backends-velox/src-delta40/test/scala/org/apache/spark/sql/delta/DeltaDeleteDeletionVectorBenchmark.scala @@ -17,12 +17,16 @@ package org.apache.spark.sql.delta import org.apache.gluten.config.{GlutenConfig, VeloxDeltaConfig} +import org.apache.gluten.execution.DeltaScanTransformer +import org.apache.gluten.extension.DeltaDeletionVectorDmlUtils +import org.apache.gluten.extension.columnar.FallbackTags import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.delta.catalog.DeltaCatalog import org.apache.spark.sql.delta.sources.DeltaSQLConf +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan} import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.util.Utils @@ -72,7 +76,18 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { filesWithDvs: Long, dvCardinality: Long, dvPayloadBytes: Long, - finalRows: Long) + finalRows: Long, + finalIdSum: BigInt, + deleteMs: Long, + validationMs: Long, + planSummary: PlanSummary) + + private case class PlanSummary( + deletePlans: Int, + glutenDeleteCommands: Int, + deltaScanTransformers: Int, + dmlRowIndexFallbackScans: Int, + fallbackReasons: Seq[String]) private var sparkSession: SparkSession = _ private var benchmarkRoot: File = _ @@ -220,6 +235,7 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { expectedDeletedMods: Seq[Int]): Unit = { val paths = prepareTables(s"$name-${mode.label}", conf, existingDv) val expectedFinalRows = expectedRemainingRows(conf.rowCount, expectedDeletedMods) + val expectedFinalIdSum = expectedRemainingIdSum(conf.rowCount, expectedDeletedMods) val benchmark = new Benchmark( name = s"$name ${mode.label} (${conf.rowCount} rows, ${conf.files} files)", valuesPerIteration = conf.rowCount, @@ -236,7 +252,7 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { paths(iteration), measuredPredicate, mode.deleteConfs) - validateDeleteResult(result, existingDv, expectedFinalRows) + validateDeleteResult(result, existingDv, expectedFinalRows, expectedFinalIdSum) printFirstIterationResult(iteration, mode.label, result) } @@ -264,7 +280,9 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { validateDeleteResult( result, existingDv = false, - expectedFinalRows = expectedRemainingRows(conf.rowCount, Seq(9))) + expectedFinalRows = expectedRemainingRows(conf.rowCount, Seq(9)), + expectedFinalIdSum = expectedRemainingIdSum(conf.rowCount, Seq(9)) + ) } path } @@ -296,33 +314,76 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { DeltaSQLConf.DELETE_USE_PERSISTENT_DELETION_VECTORS.key -> "true", DeltaConfigs.ENABLE_DELETION_VECTORS_CREATION.defaultTablePropertyKey -> "true" ) { - spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + val deleteStartNs = System.nanoTime() + val executedPlans = DeltaTestUtils.withAllPlansCaptured(spark) { + spark.sql(s"DELETE FROM delta.`$path` WHERE $predicate").collect() + }.map(_.executedPlan) + val deleteMs = elapsedMs(deleteStartNs) + collectDeleteResult(path, deleteMs, summarizePlans(executedPlans)) } - collectDeleteResult(path) } - private def collectDeleteResult(path: String): DeleteResult = { + private def collectDeleteResult( + path: String, + deleteMs: Long, + planSummary: PlanSummary): DeleteResult = { + val validationStartNs = System.nanoTime() val files = DeltaLog.forTable(spark, path).update().allFiles.collect() val filesWithDvs = files.filter(_.deletionVector != null) - val finalRows = spark.read.format("delta").load(path).count() + val finalStats = spark.read + .format("delta") + .load(path) + .selectExpr( + "count(*) as final_rows", + "coalesce(sum(cast(id as decimal(38,0))), cast(0 as decimal(38,0))) as final_id_sum") + .head() DeleteResult( activeFiles = files.length, filesWithDvs = filesWithDvs.length, dvCardinality = filesWithDvs.map(_.deletionVector.cardinality).sum, dvPayloadBytes = filesWithDvs.map(_.deletionVector.sizeInBytes).sum, - finalRows = finalRows + finalRows = finalStats.getLong(0), + finalIdSum = BigInt(finalStats.getDecimal(1).toBigInteger), + deleteMs = deleteMs, + validationMs = elapsedMs(validationStartNs), + planSummary = planSummary + ) + } + + private def summarizePlans(executedPlans: Seq[SparkPlan]): PlanSummary = { + val planNodes = executedPlans.flatMap(_.collect { case node: SparkPlan => node }) + val fileScans = planNodes.collect { case scan: FileSourceScanExec => scan } + val fallbackReasons = + planNodes.flatMap(plan => FallbackTags.getOption(plan).map(_.reason())).distinct.sorted + val dmlFallbackScans = fileScans.count { + scan => + DeltaDeletionVectorDmlUtils.isDeletionVectorDmlRowIndexScan(scan) && + FallbackTags + .getOption(scan) + .exists(_.reason().contains("fallback Delta DV DML row-index scan")) + } + PlanSummary( + deletePlans = executedPlans.length, + glutenDeleteCommands = planNodes.count(_.nodeName.contains("GlutenDeleteCommand")), + deltaScanTransformers = planNodes.count(_.isInstanceOf[DeltaScanTransformer]), + dmlRowIndexFallbackScans = dmlFallbackScans, + fallbackReasons = fallbackReasons ) } private def validateDeleteResult( result: DeleteResult, existingDv: Boolean, - expectedFinalRows: Long): Unit = { + expectedFinalRows: Long, + expectedFinalIdSum: BigInt): Unit = { require(result.filesWithDvs > 0, s"Expected deletion vectors, got $result") require(result.dvCardinality > 0, s"Expected deleted-row cardinality, got $result") require( result.finalRows == expectedFinalRows, s"Expected $expectedFinalRows final rows, got $result") + require( + result.finalIdSum == expectedFinalIdSum, + s"Expected final id sum $expectedFinalIdSum, got $result") if (existingDv) { require( result.dvCardinality > result.filesWithDvs, @@ -340,7 +401,15 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { s"filesWithDvs=${result.filesWithDvs}, " + s"dvCardinality=${result.dvCardinality}, " + s"dvPayloadBytes=${result.dvPayloadBytes}, " + - s"finalRows=${result.finalRows}") + s"finalRows=${result.finalRows}, " + + s"finalIdSum=${result.finalIdSum}, " + + s"deleteMs=${result.deleteMs}, " + + s"validationMs=${result.validationMs}, " + + s"deletePlans=${result.planSummary.deletePlans}, " + + s"glutenDeleteCommands=${result.planSummary.glutenDeleteCommands}, " + + s"deltaScanTransformers=${result.planSummary.deltaScanTransformers}, " + + s"dmlRowIndexFallbackScans=${result.planSummary.dmlRowIndexFallbackScans}, " + + s"fallbackReasons=${result.planSummary.fallbackReasons.mkString("[", "; ", "]")}") } } @@ -372,6 +441,21 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { private def expectedRemainingRows(rowCount: Long, deletedMods: Seq[Int]): Long = rowCount - deletedMods.distinct.map(countRowsWithMod(rowCount, _)).sum + private def expectedRemainingIdSum(rowCount: Long, deletedMods: Seq[Int]): BigInt = + sumRange(rowCount) - deletedMods.distinct.map(sumRowsWithMod(rowCount, _)).sum + + private def sumRange(rowCount: Long): BigInt = + BigInt(rowCount) * BigInt(rowCount - 1L) / 2 + + private def sumRowsWithMod(rowCount: Long, mod: Int): BigInt = { + val count = countRowsWithMod(rowCount, mod) + if (count == 0L) { + BigInt(0) + } else { + BigInt(count) * (BigInt(2L * mod) + BigInt(10L) * BigInt(count - 1L)) / 2 + } + } + private def countRowsWithMod(rowCount: Long, mod: Int): Long = { require(mod >= 0 && mod < 10, s"Expected modulo in [0, 10), got $mod") if (rowCount <= mod) { @@ -390,4 +474,7 @@ object DeltaDeleteDeletionVectorBenchmark extends BenchmarkBase { private def sanitize(name: String): String = name.toLowerCase(Locale.ROOT).replaceAll("[^a-z0-9]+", "-") + + private def elapsedMs(startNs: Long): Long = + (System.nanoTime() - startNs) / (1000L * 1000L) }