Skip to content

Commit

Permalink
[GLUTEN-8872][CH][Part-2] Support Delta Deletion Vectors read for CH …
Browse files Browse the repository at this point in the history
…backend

Part-2:
Support reading deletion vectors bitmap when queryring
  • Loading branch information
zzcclp committed Mar 12, 2025
1 parent 82e4eee commit 8d8ad90
Show file tree
Hide file tree
Showing 26 changed files with 670 additions and 85 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,59 @@ package org.apache.gluten.sql.shims.delta32
import org.apache.gluten.execution.GlutenPlan
import org.apache.gluten.sql.shims.DeltaShims

import org.apache.spark.sql.delta.DeltaParquetFileFormat
import org.apache.spark.sql.delta.actions.DeletionVectorDescriptor
import org.apache.spark.sql.delta.util.JsonUtils
import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.perf.DeltaOptimizedWriterTransformer

import org.apache.hadoop.fs.Path

import java.util.{HashMap => JHashMap, Map => JMap}

import scala.collection.JavaConverters._

class Delta32Shims extends DeltaShims {
override def supportDeltaOptimizedWriterExec(plan: SparkPlan): Boolean =
DeltaOptimizedWriterTransformer.support(plan)

override def offloadDeltaOptimizedWriterExec(plan: SparkPlan): GlutenPlan = {
DeltaOptimizedWriterTransformer.from(plan)
}

/**
* decode ZeroMQ Base85 encoded file path
*
* TODO: native size needs to support the ZeroMQ Base85
*/
override def convertRowIndexFilterIdEncoded(
partitionColsCnt: Int,
file: PartitionedFile,
otherConstantMetadataColumnValues: JMap[String, Object]): JMap[String, Object] = {
val newOtherConstantMetadataColumnValues: JMap[String, Object] =
new JHashMap[String, Object]
for ((k, v) <- otherConstantMetadataColumnValues.asScala) {
if (k.equalsIgnoreCase(DeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_ID_ENCODED)) {
val decoded = JsonUtils.fromJson[DeletionVectorDescriptor](v.toString)
var filePath = new Path(file.filePath.toString()).getParent
for (_ <- 0 until partitionColsCnt) {
filePath = filePath.getParent
}
val decodedPath = decoded.absolutePath(filePath)
val newDeletionVectorDescriptor = decoded.copy(
decoded.storageType,
decodedPath.toUri.toASCIIString,
decoded.offset,
decoded.sizeInBytes,
decoded.cardinality,
decoded.maxRowIndex
)
newOtherConstantMetadataColumnValues.put(k, JsonUtils.toJson(newDeletionVectorDescriptor))
} else {
newOtherConstantMetadataColumnValues.put(k, v)
}
}
newOtherConstantMetadataColumnValues
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,184 @@
/*
* 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.gluten.delta

import org.apache.gluten.execution.{FileSourceScanExecTransformer, GlutenClickHouseTPCHAbstractSuite}

import org.apache.spark.SparkConf
import org.apache.spark.sql.delta.files.TahoeFileIndex
import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper

// Some sqls' line length exceeds 100
// scalastyle:off line.size.limit

class GlutenDeltaParquetDeletionVectorSuite
extends GlutenClickHouseTPCHAbstractSuite
with AdaptiveSparkPlanHelper {

override protected val needCopyParquetToTablePath = true

override protected val tablesPath: String = basePath + "/tpch-data"
override protected val tpchQueries: String = rootPath + "queries/tpch-queries-ch"
override protected val queriesResults: String = rootPath + "mergetree-queries-output"

// import org.apache.gluten.backendsapi.clickhouse.CHConfig._

/** Run Gluten + ClickHouse Backend with SortShuffleManager */
override protected def sparkConf: SparkConf = {
super.sparkConf
.set("spark.shuffle.manager", "org.apache.spark.shuffle.sort.ColumnarShuffleManager")
.set("spark.io.compression.codec", "LZ4")
.set("spark.sql.shuffle.partitions", "5")
.set("spark.sql.autoBroadcastJoinThreshold", "10MB")
.set("spark.sql.adaptive.enabled", "true")
.set("spark.sql.files.maxPartitionBytes", "20000000")
.set("spark.sql.storeAssignmentPolicy", "legacy")
// .setCHConfig("use_local_format", true)
.set("spark.databricks.delta.retentionDurationCheck.enabled", "false")
}

override protected def createTPCHNotNullTables(): Unit = {
createNotNullTPCHTablesInParquet(tablesPath)
}

private val q1SchemaString: String =
s""" l_orderkey bigint,
| l_partkey bigint,
| l_suppkey bigint,
| l_linenumber bigint,
| l_quantity double,
| l_extendedprice double,
| l_discount double,
| l_tax double,
| l_returnflag string,
| l_linestatus string,
| l_shipdate date,
| l_commitdate date,
| l_receiptdate date,
| l_shipinstruct string,
| l_shipmode string,
| l_comment string""".stripMargin

test("test parquet table delete with the delta DV") {
spark.sql(s"""
|set spark.gluten.enabled=false;
|""".stripMargin)
spark.sql(s"""
|DROP TABLE IF EXISTS lineitem_delta_parquet_delete_dv;
|""".stripMargin)

spark.sql(s"""
|CREATE TABLE IF NOT EXISTS lineitem_delta_parquet_delete_dv
|($q1SchemaString)
|USING delta
|TBLPROPERTIES (delta.enableDeletionVectors='true')
|LOCATION '$basePath/lineitem_delta_parquet_delete_dv'
|""".stripMargin)

spark.sql(s"""
| insert into table lineitem_delta_parquet_delete_dv
| select /*+ REPARTITION(6) */ * from lineitem
|""".stripMargin)

val df1 = spark.sql(s"""
| delete from lineitem_delta_parquet_delete_dv
| where mod(l_orderkey, 3) = 1 and l_orderkey < 100
|""".stripMargin)
spark.sql(s"""
|set spark.gluten.enabled=true;
|""".stripMargin)

val df = spark.sql(s"""
| select sum(l_linenumber) from lineitem_delta_parquet_delete_dv
|""".stripMargin)
val result = df.collect()
assert(
result.apply(0).get(0) === 1802335
)
val scanExec = collect(df.queryExecution.executedPlan) {
case f: FileSourceScanExecTransformer => f
}
val parquetScan = scanExec.head
val fileIndex = parquetScan.relation.location.asInstanceOf[TahoeFileIndex]
val addFiles = fileIndex.matchingFiles(Nil, Nil)
assert(addFiles.size === 6)

spark.sql(s"""
|set spark.gluten.enabled=false;
|""".stripMargin)
spark.sql(s"""
| delete from lineitem_delta_parquet_delete_dv where mod(l_orderkey, 3) = 2
|""".stripMargin)
spark.sql(s"""
|set spark.gluten.enabled=true;
|""".stripMargin)

val df3 = spark.sql(s"""
| select sum(l_linenumber) from lineitem_delta_parquet_delete_dv
|""".stripMargin)
assert(
df3.collect().apply(0).get(0) === 1200560
)
}

test("test parquet partition table delete with the delta DV") {
withSQLConf(("spark.sql.sources.partitionOverwriteMode", "dynamic")) {
spark.sql(s"""
|set spark.gluten.enabled=false;
|""".stripMargin)
spark.sql(s"""
|DROP TABLE IF EXISTS lineitem_delta_partition_parquet_delete_dv;
|""".stripMargin)

spark.sql(s"""
|CREATE TABLE IF NOT EXISTS lineitem_delta_partition_parquet_delete_dv
|($q1SchemaString)
|USING delta
|PARTITIONED BY (l_returnflag)
|TBLPROPERTIES (delta.enableDeletionVectors='true')
|LOCATION '$basePath/lineitem_delta_partition_parquet_delete_dv'
|""".stripMargin)

spark.sql(s"""
| insert into table lineitem_delta_partition_parquet_delete_dv
| select /*+ REPARTITION(6) */ * from lineitem
|""".stripMargin)

val df1 = spark.sql(s"""
| delete from lineitem_delta_partition_parquet_delete_dv
| where mod(l_orderkey, 3) = 1
|""".stripMargin)
spark.sql(s"""
|set spark.gluten.enabled=true;
|""".stripMargin)

val df =
spark.sql(s"""
| select sum(l_linenumber) from lineitem_delta_partition_parquet_delete_dv
|""".stripMargin)
val result = df.collect()
assert(
result.apply(0).get(0) === 1201486
)
val scanExec = collect(df.queryExecution.executedPlan) {
case f: FileSourceScanExecTransformer => f
}
assert(scanExec.nonEmpty)
}
}
}
// scalastyle:off line.size.limit
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,9 @@ package org.apache.gluten.sql.shims
import org.apache.gluten.execution.GlutenPlan

import org.apache.spark.sql.execution.SparkPlan
import org.apache.spark.sql.execution.datasources.PartitionedFile

import java.util.{HashMap => JHashMap, Map => JMap}

sealed abstract class ShimDescriptor

Expand All @@ -29,4 +32,10 @@ trait DeltaShims {
throw new UnsupportedOperationException(
s"Can't transform ColumnarDeltaOptimizedWriterExec from ${plan.getClass.getSimpleName}")
}

def convertRowIndexFilterIdEncoded(
partitionColsCnt: Int,
file: PartitionedFile,
otherConstantMetadataColumnValues: JMap[String, Object]): JMap[String, Object] =
new JHashMap[String, Object]()
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.gluten.execution._
import org.apache.gluten.expression.ConverterUtils
import org.apache.gluten.logging.LogLevelUtil
import org.apache.gluten.metrics.IMetrics
import org.apache.gluten.sql.shims.SparkShimLoader
import org.apache.gluten.sql.shims.{DeltaShimLoader, SparkShimLoader}
import org.apache.gluten.substrait.plan.PlanNode
import org.apache.gluten.substrait.rel._
import org.apache.gluten.substrait.rel.LocalFilesNode.ReadFileFormat
Expand Down Expand Up @@ -159,6 +159,7 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
val modificationTimes = new JArrayList[JLong]()
val partitionColumns = new JArrayList[JMap[String, String]]
val metadataColumns = new JArrayList[JMap[String, String]]
val otherMetadataColumns = new JArrayList[JMap[String, Object]]
f.files.foreach {
file =>
paths.add(new URI(file.filePath.toString()).toASCIIString)
Expand Down Expand Up @@ -203,6 +204,14 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
fileSizes.add(0)
modificationTimes.add(0)
}

val otherConstantMetadataColumnValues =
DeltaShimLoader.getDeltaShims.convertRowIndexFilterIdEncoded(
partitionColumn.size(),
file,
SparkShimLoader.getSparkShims.getOtherConstantMetadataColumnValues(file)
)
otherMetadataColumns.add(otherConstantMetadataColumnValues)
}
val preferredLocations =
CHAffinity.getFilePartitionLocations(paths.asScala.toArray, f.preferredLocations())
Expand All @@ -217,7 +226,8 @@ class CHIteratorApi extends IteratorApi with Logging with LogLevelUtil {
metadataColumns,
fileFormat,
preferredLocations.toList.asJava,
mapAsJavaMap(properties)
mapAsJavaMap(properties),
otherMetadataColumns
)
case _ =>
throw new UnsupportedOperationException(s"Unsupported input partition: $partition.")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,18 +70,19 @@ case class CHInputPartitionsUtil(
SparkShimLoader.getSparkShims.getFileStatus(partition).flatMap {
file =>
// getPath() is very expensive so we only want to call it once in this block:
val filePath = file.getPath
val filePath = file._1.getPath

if (shouldProcess(filePath)) {
val isSplitable =
relation.fileFormat.isSplitable(relation.sparkSession, relation.options, filePath)
SparkShimLoader.getSparkShims.splitFiles(
relation.sparkSession,
file,
file._1,
filePath,
isSplitable,
maxSplitBytes,
partition.values
partition.values,
file._2
)
} else {
Seq.empty
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -121,7 +121,8 @@ case class GlutenCacheFilesCommand(
new JArrayList[JMap[String, String]](),
ReadFileFormat.ParquetReadFormat, // ignore format in backend
new JArrayList[String](),
new JHashMap[String, String]()
new JHashMap[String, String](),
new JArrayList[JMap[String, Object]]()
)

(executorId, localFile)
Expand Down
Loading

0 comments on commit 8d8ad90

Please sign in to comment.