Skip to content

Commit be7183b

Browse files
authored
[Spark] DV Reads Performance Improvement in Delta by removing Broadcasting DV Information (#2888)
<!-- Thanks for sending a pull request! Here are some tips for you: 1. If this is your first time, please read our contributor guidelines: https://github.com/delta-io/delta/blob/master/CONTRIBUTING.md 2. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP] Your PR title ...'. 3. Be sure to keep the PR description updated to reflect all changes. 4. Please write your PR title to summarize what this PR proposes. 5. If possible, provide a concise example to reproduce the issue for a faster review. 6. If applicable, include the corresponding issue number in the PR title and link it in the body. --> #### Which Delta project/connector is this regarding? <!-- Please add the component selected below to the beginning of the pull request title For example: [Spark] Title of my pull request --> - [X] Spark - [ ] Standalone - [ ] Flink - [ ] Kernel - [ ] Other (fill in here) ## Description <!-- - Describe what this PR changes. - Describe why we need the change. If this PR resolves an issue be sure to include "Resolves #XXX" to correctly link and close the issue upon merge. --> Back then, we relied on an [expensive Broadcast of DV files](#1542) to pass the DV files to the associated Parquet Files. With the introduction of [adding custom metadata to files](apache/spark#40677) introduced in Spark 3.5, we can now pass the DV through the custom metadata field, this is expected to improve the performance of DV reads in Delta. ## How was this patch tested? <!-- If tests were added, say they were added here. Please make sure to test the changes thoroughly including negative and positive cases if possible. If the changes were tested in any way other than unit tests, please clarify how you tested step by step (ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future). If the changes were not tested, please explain why. --> Adjusted the existing UTs that cover our changes. ## Does this PR introduce _any_ user-facing changes? No. <!-- If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible. If possible, please also clarify if this is a user-facing change compared to the released Delta Lake versions or within the unreleased branches such as master. If no, write 'No'. -->
1 parent cb53c9a commit be7183b

File tree

4 files changed

+167
-212
lines changed

4 files changed

+167
-212
lines changed

spark/src/main/scala/org/apache/spark/sql/delta/DeltaParquetFileFormat.scala

Lines changed: 45 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -16,8 +16,6 @@
1616

1717
package org.apache.spark.sql.delta
1818

19-
import java.net.URI
20-
2119
import scala.collection.mutable.ArrayBuffer
2220
import scala.util.control.NonFatal
2321

@@ -32,17 +30,17 @@ import org.apache.hadoop.mapreduce.Job
3230
import org.apache.parquet.hadoop.ParquetOutputFormat
3331
import org.apache.parquet.hadoop.util.ContextUtil
3432

35-
import org.apache.spark.broadcast.Broadcast
3633
import org.apache.spark.sql.SparkSession
3734
import org.apache.spark.sql.catalyst.InternalRow
35+
import org.apache.spark.sql.catalyst.expressions.FileSourceConstantMetadataStructField
3836
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits.parseColumnPath
3937
import org.apache.spark.sql.execution.datasources.OutputWriterFactory
4038
import org.apache.spark.sql.execution.datasources.PartitionedFile
4139
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
4240
import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
4341
import org.apache.spark.sql.internal.SQLConf
4442
import org.apache.spark.sql.sources._
45-
import org.apache.spark.sql.types.{ByteType, LongType, MetadataBuilder, StructField, StructType}
43+
import org.apache.spark.sql.types.{ByteType, LongType, MetadataBuilder, StringType, StructField, StructType}
4644
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnarBatchRow, ColumnVector}
4745
import org.apache.spark.util.SerializableConfiguration
4846

@@ -59,13 +57,11 @@ case class DeltaParquetFileFormat(
5957
nullableRowTrackingFields: Boolean = false,
6058
isSplittable: Boolean = true,
6159
disablePushDowns: Boolean = false,
62-
tablePath: Option[String] = None,
63-
broadcastDvMap: Option[Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]]] = None,
64-
broadcastHadoopConf: Option[Broadcast[SerializableConfiguration]] = None)
60+
tablePath: Option[String] = None)
6561
extends ParquetFileFormat {
6662
// Validate either we have all arguments for DV enabled read or none of them.
67-
if (hasDeletionVectorMap) {
68-
require(tablePath.isDefined && !isSplittable && disablePushDowns,
63+
if (hasTablePath) {
64+
require(!isSplittable && disablePushDowns,
6965
"Wrong arguments for Delta table scan with deletion vectors")
7066
}
7167

@@ -124,7 +120,7 @@ case class DeltaParquetFileFormat(
124120
override def isSplitable(
125121
sparkSession: SparkSession, options: Map[String, String], path: Path): Boolean = isSplittable
126122

127-
def hasDeletionVectorMap: Boolean = broadcastDvMap.isDefined && broadcastHadoopConf.isDefined
123+
def hasTablePath: Boolean = tablePath.isDefined
128124

129125
/**
130126
* We sometimes need to replace FileFormat within LogicalPlans, so we have to override
@@ -182,11 +178,13 @@ case class DeltaParquetFileFormat(
182178
require(disablePushDowns, "Cannot generate row index related metadata with filter pushdown")
183179
}
184180

185-
if (hasDeletionVectorMap && isRowDeletedColumn.isEmpty) {
181+
if (hasTablePath && isRowDeletedColumn.isEmpty) {
186182
throw new IllegalArgumentException(
187183
s"Expected a column $IS_ROW_DELETED_COLUMN_NAME in the schema")
188184
}
189185

186+
val serializableHadoopConf = new SerializableConfiguration(hadoopConf)
187+
190188
val useOffHeapBuffers = sparkSession.sessionState.conf.offHeapColumnVectorEnabled
191189
(partitionedFile: PartitionedFile) => {
192190
val rowIteratorFromParquet = parquetDataReader(partitionedFile)
@@ -196,8 +194,9 @@ case class DeltaParquetFileFormat(
196194
partitionedFile,
197195
rowIteratorFromParquet,
198196
isRowDeletedColumn,
199-
useOffHeapBuffers = useOffHeapBuffers,
200-
rowIndexColumn = rowIndexColumn)
197+
rowIndexColumn,
198+
useOffHeapBuffers,
199+
serializableHadoopConf)
201200
iterToReturn.asInstanceOf[Iterator[InternalRow]]
202201
} catch {
203202
case NonFatal(e) =>
@@ -276,16 +275,11 @@ case class DeltaParquetFileFormat(
276275
.updated(DefaultRowCommitVersion.METADATA_STRUCT_FIELD_NAME, extractDefaultRowCommitVersion)
277276
}
278277

279-
def copyWithDVInfo(
280-
tablePath: String,
281-
broadcastDvMap: Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]],
282-
broadcastHadoopConf: Broadcast[SerializableConfiguration]): DeltaParquetFileFormat = {
278+
def disableSplittingAndPushdown(tablePath: String): DeltaParquetFileFormat = {
283279
this.copy(
284280
isSplittable = false,
285281
disablePushDowns = true,
286-
tablePath = Some(tablePath),
287-
broadcastDvMap = Some(broadcastDvMap),
288-
broadcastHadoopConf = Some(broadcastHadoopConf))
282+
tablePath = Some(tablePath))
289283
}
290284

291285
/**
@@ -300,28 +294,32 @@ case class DeltaParquetFileFormat(
300294
iterator: Iterator[Object],
301295
isRowDeletedColumn: Option[ColumnMetadata],
302296
rowIndexColumn: Option[ColumnMetadata],
303-
useOffHeapBuffers: Boolean): Iterator[Object] = {
304-
val pathUri = partitionedFile.pathUri
305-
297+
useOffHeapBuffers: Boolean,
298+
serializableHadoopConf: SerializableConfiguration): Iterator[Object] = {
306299
val rowIndexFilter = isRowDeletedColumn.map { col =>
307300
// Fetch the DV descriptor from the broadcast map and create a row index filter
308-
broadcastDvMap.get.value
309-
.get(pathUri)
310-
.map { case DeletionVectorDescriptorWithFilterType(dvDescriptor, filterType) =>
311-
filterType match {
312-
case i if i == RowIndexFilterType.IF_CONTAINED =>
313-
DropMarkedRowsFilter.createInstance(
314-
dvDescriptor,
315-
broadcastHadoopConf.get.value.value,
316-
tablePath.map(new Path(_)))
317-
case i if i == RowIndexFilterType.IF_NOT_CONTAINED =>
318-
KeepMarkedRowsFilter.createInstance(
319-
dvDescriptor,
320-
broadcastHadoopConf.get.value.value,
321-
tablePath.map(new Path(_)))
322-
}
301+
val dvDescriptorOpt = partitionedFile.otherConstantMetadataColumnValues
302+
.get(FILE_ROW_INDEX_FILTER_ID_ENCODED)
303+
val filterTypeOpt = partitionedFile.otherConstantMetadataColumnValues
304+
.get(FILE_ROW_INDEX_FILTER_TYPE)
305+
if (dvDescriptorOpt.isDefined && filterTypeOpt.isDefined) {
306+
val rowIndexFilter = filterTypeOpt.get match {
307+
case RowIndexFilterType.IF_CONTAINED => DropMarkedRowsFilter
308+
case RowIndexFilterType.IF_NOT_CONTAINED => KeepMarkedRowsFilter
309+
case unexpectedFilterType => throw new IllegalStateException(
310+
s"Unexpected row index filter type: ${unexpectedFilterType}")
323311
}
324-
.getOrElse(KeepAllRowsFilter)
312+
rowIndexFilter.createInstance(
313+
DeletionVectorDescriptor.fromJson(dvDescriptorOpt.get.asInstanceOf[String]),
314+
serializableHadoopConf.value,
315+
tablePath.map(new Path(_)))
316+
} else if (dvDescriptorOpt.isDefined || filterTypeOpt.isDefined) {
317+
throw new IllegalStateException(
318+
s"Both ${FILE_ROW_INDEX_FILTER_ID_ENCODED} and ${FILE_ROW_INDEX_FILTER_TYPE} " +
319+
"should either both have values or no values at all.")
320+
} else {
321+
KeepAllRowsFilter
322+
}
325323
}
326324

327325
val metadataColumns = Seq(isRowDeletedColumn, rowIndexColumn).filter(_.nonEmpty).map(_.get)
@@ -417,6 +415,14 @@ object DeltaParquetFileFormat {
417415
val ROW_INDEX_COLUMN_NAME = "__delta_internal_row_index"
418416
val ROW_INDEX_STRUCT_FIELD = StructField(ROW_INDEX_COLUMN_NAME, LongType)
419417

418+
/** The key to the encoded row index filter identifier value of the
419+
* [[PartitionedFile]]'s otherConstantMetadataColumnValues map. */
420+
val FILE_ROW_INDEX_FILTER_ID_ENCODED = "row_index_filter_id_encoded"
421+
422+
/** The key to the row index filter type value of the
423+
* [[PartitionedFile]]'s otherConstantMetadataColumnValues map. */
424+
val FILE_ROW_INDEX_FILTER_TYPE = "row_index_filter_type"
425+
420426
/** Utility method to create a new writable vector */
421427
private def newVector(
422428
useOffHeapBuffers: Boolean, size: Int, dataType: StructField): WritableColumnVector = {
@@ -484,11 +490,6 @@ object DeltaParquetFileFormat {
484490
/** Helper class to encapsulate column info */
485491
case class ColumnMetadata(index: Int, structField: StructField)
486492

487-
/** Helper class that encapsulate an [[RowIndexFilterType]]. */
488-
case class DeletionVectorDescriptorWithFilterType(
489-
descriptor: DeletionVectorDescriptor,
490-
filterType: RowIndexFilterType)
491-
492493
/**
493494
* Translates the filter to use physical column names instead of logical column names.
494495
* This is needed when the column mapping mode is set to `NameMapping` or `IdMapping`

spark/src/main/scala/org/apache/spark/sql/delta/PreprocessTableWithDVs.scala

Lines changed: 9 additions & 53 deletions
Original file line numberDiff line numberDiff line change
@@ -16,23 +16,16 @@
1616

1717
package org.apache.spark.sql.delta
1818

19-
import java.net.URI
20-
21-
import org.apache.spark.sql.delta.{RowIndexFilter, RowIndexFilterType}
19+
import org.apache.spark.sql.delta.RowIndexFilter
2220
import org.apache.spark.sql.delta.DeltaParquetFileFormat._
2321
import org.apache.spark.sql.delta.commands.DeletionVectorUtils.deletionVectorsReadable
2422
import org.apache.spark.sql.delta.files.{TahoeFileIndex, TahoeLogFileIndex}
25-
import org.apache.spark.sql.delta.util.DeltaFileOperations.absolutePath
2623

27-
import org.apache.spark.broadcast.Broadcast
28-
import org.apache.spark.sql.{Column, SparkSession}
24+
import org.apache.spark.sql.Column
2925
import org.apache.spark.sql.catalyst.expressions.AttributeReference
3026
import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
3127
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project}
32-
import org.apache.spark.sql.catalyst.rules.Rule
3328
import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation}
34-
import org.apache.spark.sql.types.StructType
35-
import org.apache.spark.util.SerializableConfiguration
3629

3730
/**
3831
* Plan transformer to inject a filter that removes the rows marked as deleted according to
@@ -54,8 +47,6 @@ import org.apache.spark.util.SerializableConfiguration
5447
* to generate the row index. This is a cost we need to pay until we upgrade to latest
5548
* Apache Spark which contains Parquet reader changes that automatically generate the
5649
* row_index irrespective of the file splitting and filter pushdowns.
57-
* - The scan created also contains a broadcast variable of Parquet File -> DV File map.
58-
* The Parquet reader created uses this map to find the DV file corresponding to the data file.
5950
* - Filter created filters out rows with __skip_row equals to 0
6051
* - And at the end we have a Project to keep the plan node output same as before the rule is
6152
* applied.
@@ -93,20 +84,20 @@ object ScanWithDeletionVectors {
9384

9485
// See if the relation is already modified to include DV reads as part of
9586
// a previous invocation of this rule on this table
96-
if (fileFormat.hasDeletionVectorMap) return None
87+
if (fileFormat.hasTablePath) return None
9788

9889
// See if any files actually have a DV
99-
val spark = SparkSession.getActiveSession.get
100-
val filePathToDVBroadcastMap = createBroadcastDVMap(spark, index)
101-
if (filePathToDVBroadcastMap.value.isEmpty) return None
90+
val filesWithDVs = index
91+
.matchingFiles(partitionFilters = Seq(TrueLiteral), dataFilters = Seq(TrueLiteral))
92+
.filter(_.deletionVector != null)
93+
if (filesWithDVs.isEmpty) return None
10294

10395
// Get the list of columns in the output of the `LogicalRelation` we are
10496
// trying to modify. At the end of the plan, we need to return a
10597
// `LogicalRelation` that has the same output as this `LogicalRelation`
10698
val planOutput = scan.output
10799

108-
val newScan = createScanWithSkipRowColumn(
109-
spark, scan, fileFormat, index, filePathToDVBroadcastMap, hadoopRelation)
100+
val newScan = createScanWithSkipRowColumn(scan, fileFormat, index, hadoopRelation)
110101

111102
// On top of the scan add a filter that filters out the rows which have
112103
// skip row column value non-zero
@@ -121,11 +112,9 @@ object ScanWithDeletionVectors {
121112
* an extra column which indicates whether the row needs to be skipped or not.
122113
*/
123114
private def createScanWithSkipRowColumn(
124-
spark: SparkSession,
125115
inputScan: LogicalRelation,
126116
fileFormat: DeltaParquetFileFormat,
127117
tahoeFileIndex: TahoeFileIndex,
128-
filePathToDVBroadcastMap: Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]],
129118
hadoopFsRelation: HadoopFsRelation): LogicalRelation = {
130119
// Create a new `LogicalRelation` that has modified `DeltaFileFormat` and output with an extra
131120
// column to indicate whether to skip the row or not
@@ -141,11 +130,7 @@ object ScanWithDeletionVectors {
141130
// operator after the data is read from the underlying file reader.
142131
val newDataSchema = hadoopFsRelation.dataSchema.add(skipRowField)
143132

144-
val hadoopConfBroadcast = spark.sparkContext.broadcast(
145-
new SerializableConfiguration(tahoeFileIndex.deltaLog.newDeltaHadoopConf()))
146-
147-
val newFileFormat = fileFormat.copyWithDVInfo(
148-
tahoeFileIndex.path.toString, filePathToDVBroadcastMap, hadoopConfBroadcast)
133+
val newFileFormat = fileFormat.disableSplittingAndPushdown(tahoeFileIndex.path.toString)
149134
val newRelation = hadoopFsRelation.copy(
150135
fileFormat = newFileFormat,
151136
dataSchema = newDataSchema)(hadoopFsRelation.sparkSession)
@@ -166,33 +151,4 @@ object ScanWithDeletionVectors {
166151
val filterExp = keepRow(new Column(skipRowColumnRef)).expr
167152
Filter(filterExp, newScan)
168153
}
169-
170-
private def createBroadcastDVMap(
171-
spark: SparkSession,
172-
tahoeFileIndex: TahoeFileIndex)
173-
: Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]] = {
174-
val filterTypes = tahoeFileIndex.rowIndexFilters.getOrElse(Map.empty)
175-
176-
// Given there is no way to find the final filters, just select all files in the
177-
// file index and create the DV map.
178-
val filesWithDVs = tahoeFileIndex
179-
.matchingFiles(partitionFilters = Seq(TrueLiteral), dataFilters = Seq(TrueLiteral))
180-
.filter(_.deletionVector != null)
181-
// Attach filter types to FileActions, so that later [[DeltaParquetFileFormat]] could pick it up
182-
// to decide which kind of rows should be filtered out. This info is necessary for reading CDC
183-
// rows that have been deleted (marked in DV), in which case marked rows must be kept rather
184-
// than filtered out. In such a case, the `filterTypes` map will be populated by [[CDCReader]]
185-
// to indicate IF_NOT_CONTAINED filter should be used. In other cases, `filterTypes` will be
186-
// empty, so we generate IF_CONTAINED as the default DV behavior.
187-
val filePathToDVMap = filesWithDVs.map { addFile =>
188-
val key = absolutePath(tahoeFileIndex.path.toString, addFile.path).toUri
189-
val filterType =
190-
filterTypes.getOrElse(addFile.path, RowIndexFilterType.IF_CONTAINED)
191-
val value =
192-
DeletionVectorDescriptorWithFilterType(addFile.deletionVector, filterType)
193-
key -> value
194-
}.toMap
195-
196-
spark.sparkContext.broadcast(filePathToDVMap)
197-
}
198154
}

spark/src/main/scala/org/apache/spark/sql/delta/files/TahoeFileIndex.scala

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -22,13 +22,14 @@ import java.util.Objects
2222

2323
import scala.collection.mutable
2424
import org.apache.spark.sql.delta.RowIndexFilterType
25-
import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaErrors, DeltaLog, NoMapping, Snapshot, SnapshotDescriptor}
25+
import org.apache.spark.sql.delta.{DeltaColumnMapping, DeltaErrors, DeltaLog, DeltaParquetFileFormat, Snapshot, SnapshotDescriptor}
2626
import org.apache.spark.sql.delta.DefaultRowCommitVersion
2727
import org.apache.spark.sql.delta.RowId
2828
import org.apache.spark.sql.delta.actions.{AddFile, Metadata, Protocol}
2929
import org.apache.spark.sql.delta.implicits._
3030
import org.apache.spark.sql.delta.schema.SchemaUtils
3131
import org.apache.spark.sql.delta.sources.DeltaSQLConf
32+
import org.apache.spark.sql.delta.util.JsonUtils
3233
import org.apache.hadoop.fs.FileStatus
3334
import org.apache.hadoop.fs.Path
3435

@@ -126,6 +127,17 @@ abstract class TahoeFileIndex(
126127
addFile.defaultRowCommitVersion.foreach(defaultRowCommitVersion =>
127128
metadata.put(DefaultRowCommitVersion.METADATA_STRUCT_FIELD_NAME, defaultRowCommitVersion))
128129

130+
if (addFile.deletionVector != null) {
131+
metadata.put(DeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_ID_ENCODED,
132+
JsonUtils.toJson(addFile.deletionVector))
133+
134+
// Set the filter type to IF_CONTAINED by default to let [[DeltaParquetFileFormat]] filter
135+
// out rows unless a filter type was explicitly provided in rowIndexFilters. This can happen
136+
// e.g. when reading CDC data to keep deleted rows instead of filtering them out.
137+
val filterType = rowIndexFilters.getOrElse(Map.empty)
138+
.getOrElse(addFile.path, RowIndexFilterType.IF_CONTAINED)
139+
metadata.put(DeltaParquetFileFormat.FILE_ROW_INDEX_FILTER_TYPE, filterType)
140+
}
129141
FileStatusWithMetadata(fs, metadata.toMap)
130142
}
131143

0 commit comments

Comments
 (0)