-
Notifications
You must be signed in to change notification settings - Fork 1.6k
/
DeltaParquetFileFormat.scala
462 lines (417 loc) · 19.2 KB
/
DeltaParquetFileFormat.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
/*
* Copyright (2021) The Delta Lake Project Authors.
*
* Licensed 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 java.net.URI
import scala.collection.mutable.ArrayBuffer
import scala.util.control.NonFatal
import org.apache.spark.sql.delta.RowIndexFilterType
import org.apache.spark.sql.delta.DeltaParquetFileFormat._
import org.apache.spark.sql.delta.actions.{DeletionVectorDescriptor, Metadata, Protocol}
import org.apache.spark.sql.delta.deletionvectors.{DropMarkedRowsFilter, KeepAllRowsFilter, KeepMarkedRowsFilter}
import org.apache.spark.sql.delta.schema.SchemaMergingUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hadoop.mapreduce.Job
import org.apache.parquet.hadoop.ParquetOutputFormat
import org.apache.parquet.hadoop.util.ContextUtil
import org.apache.spark.broadcast.Broadcast
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.OutputWriterFactory
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.{ByteType, LongType, MetadataBuilder, StructField, StructType}
import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnarBatchRow, ColumnVector}
import org.apache.spark.util.SerializableConfiguration
/**
* A thin wrapper over the Parquet file format to support
* - columns names without restrictions.
* - populated a column from the deletion vector of this file (if exists) to indicate
* whether the row is deleted or not according to the deletion vector. Consumers
* of this scan can use the column values to filter out the deleted rows.
*/
case class DeltaParquetFileFormat(
protocol: Protocol,
metadata: Metadata,
isSplittable: Boolean = true,
disablePushDowns: Boolean = false,
tablePath: Option[String] = None,
broadcastDvMap: Option[Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]]] = None,
broadcastHadoopConf: Option[Broadcast[SerializableConfiguration]] = None)
extends ParquetFileFormat {
// Validate either we have all arguments for DV enabled read or none of them.
if (hasDeletionVectorMap) {
require(tablePath.isDefined && !isSplittable && disablePushDowns,
"Wrong arguments for Delta table scan with deletion vectors")
}
val columnMappingMode: DeltaColumnMappingMode = metadata.columnMappingMode
val referenceSchema: StructType = metadata.schema
if (columnMappingMode == IdMapping) {
val requiredReadConf = SQLConf.PARQUET_FIELD_ID_READ_ENABLED
require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredReadConf)),
s"${requiredReadConf.key} must be enabled to support Delta id column mapping mode")
val requiredWriteConf = SQLConf.PARQUET_FIELD_ID_WRITE_ENABLED
require(SparkSession.getActiveSession.exists(_.sessionState.conf.getConf(requiredWriteConf)),
s"${requiredWriteConf.key} must be enabled to support Delta id column mapping mode")
}
/**
* prepareSchemaForRead must only be used for parquet read.
* It removes "PARQUET_FIELD_ID_METADATA_KEY" for name mapping mode which address columns by
* physical name instead of id.
*/
def prepareSchemaForRead(inputSchema: StructType): StructType = {
val schema = DeltaColumnMapping.createPhysicalSchema(
inputSchema, referenceSchema, columnMappingMode)
if (columnMappingMode == NameMapping) {
SchemaMergingUtils.transformColumns(schema) { (_, field, _) =>
field.copy(metadata = new MetadataBuilder()
.withMetadata(field.metadata)
.remove(DeltaColumnMapping.PARQUET_FIELD_ID_METADATA_KEY)
.remove(DeltaColumnMapping.PARQUET_FIELD_NESTED_IDS_METADATA_KEY)
.build())
}
} else schema
}
override def isSplitable(
sparkSession: SparkSession, options: Map[String, String], path: Path): Boolean = isSplittable
def hasDeletionVectorMap: Boolean = broadcastDvMap.isDefined && broadcastHadoopConf.isDefined
/**
* We sometimes need to replace FileFormat within LogicalPlans, so we have to override
* `equals` to ensure file format changes are captured
*/
override def equals(other: Any): Boolean = {
other match {
case ff: DeltaParquetFileFormat =>
ff.columnMappingMode == columnMappingMode &&
ff.referenceSchema == referenceSchema &&
ff.isSplittable == isSplittable &&
ff.disablePushDowns == disablePushDowns
case _ => false
}
}
override def hashCode(): Int = getClass.getCanonicalName.hashCode()
override def buildReaderWithPartitionValues(
sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val pushdownFilters = if (disablePushDowns) Seq.empty else filters
val parquetDataReader: PartitionedFile => Iterator[InternalRow] =
super.buildReaderWithPartitionValues(
sparkSession,
prepareSchemaForRead(dataSchema),
prepareSchemaForRead(partitionSchema),
prepareSchemaForRead(requiredSchema),
pushdownFilters,
options,
hadoopConf)
val schemaWithIndices = requiredSchema.fields.zipWithIndex
def findColumn(name: String): Option[ColumnMetadata] = {
val results = schemaWithIndices.filter(_._1.name == name)
if (results.length > 1) {
throw new IllegalArgumentException(
s"There are more than one column with name=`$name` requested in the reader output")
}
results.headOption.map(e => ColumnMetadata(e._2, e._1))
}
val isRowDeletedColumn = findColumn(IS_ROW_DELETED_COLUMN_NAME)
val rowIndexColumn = findColumn(ROW_INDEX_COLUMN_NAME)
if (isRowDeletedColumn.isEmpty && rowIndexColumn.isEmpty) {
return parquetDataReader // no additional metadata is needed.
} else {
// verify the file splitting and filter pushdown are disabled. The new additional
// metadata columns cannot be generated with file splitting and filter pushdowns
require(!isSplittable, "Cannot generate row index related metadata with file splitting")
require(disablePushDowns, "Cannot generate row index related metadata with filter pushdown")
}
if (hasDeletionVectorMap && isRowDeletedColumn.isEmpty) {
throw new IllegalArgumentException(
s"Expected a column $IS_ROW_DELETED_COLUMN_NAME in the schema")
}
val useOffHeapBuffers = sparkSession.sessionState.conf.offHeapColumnVectorEnabled
(partitionedFile: PartitionedFile) => {
val rowIteratorFromParquet = parquetDataReader(partitionedFile)
try {
val iterToReturn =
iteratorWithAdditionalMetadataColumns(
partitionedFile,
rowIteratorFromParquet,
isRowDeletedColumn,
useOffHeapBuffers = useOffHeapBuffers,
rowIndexColumn = rowIndexColumn)
iterToReturn.asInstanceOf[Iterator[InternalRow]]
} catch {
case NonFatal(e) =>
// Close the iterator if it is a closeable resource. The `ParquetFileFormat` opens
// the file and returns `RecordReaderIterator` (which implements `AutoCloseable` and
// `Iterator`) instance as a `Iterator`.
rowIteratorFromParquet match {
case resource: AutoCloseable => closeQuietly(resource)
case _ => // do nothing
}
throw e
}
}
}
override def supportFieldName(name: String): Boolean = {
if (columnMappingMode != NoMapping) true else super.supportFieldName(name)
}
override def metadataSchemaFields: Seq[StructField] = {
// Parquet reader in Spark has a bug where a file containing 2b+ rows in a single rowgroup
// causes it to run out of the `Integer` range (TODO: Create a SPARK issue)
// For Delta Parquet readers don't expose the row_index field as a metadata field.
super.metadataSchemaFields.filter(field => field != ParquetFileFormat.ROW_INDEX_FIELD) ++
RowId.createBaseRowIdField(protocol, metadata) ++
DefaultRowCommitVersion.createDefaultRowCommitVersionField(protocol, metadata)
}
override def prepareWrite(
sparkSession: SparkSession,
job: Job,
options: Map[String, String],
dataSchema: StructType): OutputWriterFactory = {
val factory = super.prepareWrite(sparkSession, job, options, dataSchema)
val conf = ContextUtil.getConfiguration(job)
// Always write timestamp as TIMESTAMP_MICROS for Iceberg compat based on Iceberg spec
if (IcebergCompatV1.isEnabled(metadata) || IcebergCompatV2.isEnabled(metadata)) {
conf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key,
SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS.toString)
}
if (IcebergCompatV2.isEnabled(metadata)) {
// For Uniform with IcebergCompatV2, we need to write nested field IDs for list and map
// types to the parquet schema. Spark currently does not support it so we hook in our
// own write support class.
ParquetOutputFormat.setWriteSupportClass(job, classOf[DeltaParquetWriteSupport])
}
factory
}
override def fileConstantMetadataExtractors: Map[String, PartitionedFile => Any] = {
val extractBaseRowId: PartitionedFile => Any = { file =>
file.otherConstantMetadataColumnValues.getOrElse(RowId.BASE_ROW_ID, {
throw new IllegalStateException(
s"Missing ${RowId.BASE_ROW_ID} value for file '${file.filePath}'")
})
}
val extractDefaultRowCommitVersion: PartitionedFile => Any = { file =>
file.otherConstantMetadataColumnValues
.getOrElse(DefaultRowCommitVersion.METADATA_STRUCT_FIELD_NAME, {
throw new IllegalStateException(
s"Missing ${DefaultRowCommitVersion.METADATA_STRUCT_FIELD_NAME} value " +
s"for file '${file.filePath}'")
})
}
super.fileConstantMetadataExtractors
.updated(RowId.BASE_ROW_ID, extractBaseRowId)
.updated(DefaultRowCommitVersion.METADATA_STRUCT_FIELD_NAME, extractDefaultRowCommitVersion)
}
def copyWithDVInfo(
tablePath: String,
broadcastDvMap: Broadcast[Map[URI, DeletionVectorDescriptorWithFilterType]],
broadcastHadoopConf: Broadcast[SerializableConfiguration]): DeltaParquetFileFormat = {
this.copy(
isSplittable = false,
disablePushDowns = true,
tablePath = Some(tablePath),
broadcastDvMap = Some(broadcastDvMap),
broadcastHadoopConf = Some(broadcastHadoopConf))
}
/**
* Modifies the data read from underlying Parquet reader by populating one or both of the
* following metadata columns.
* - [[IS_ROW_DELETED_COLUMN_NAME]] - row deleted status from deletion vector corresponding
* to this file
* - [[ROW_INDEX_COLUMN_NAME]] - index of the row within the file.
*/
private def iteratorWithAdditionalMetadataColumns(
partitionedFile: PartitionedFile,
iterator: Iterator[Object],
isRowDeletedColumn: Option[ColumnMetadata],
rowIndexColumn: Option[ColumnMetadata],
useOffHeapBuffers: Boolean): Iterator[Object] = {
val pathUri = partitionedFile.pathUri
val rowIndexFilter = isRowDeletedColumn.map { col =>
// Fetch the DV descriptor from the broadcast map and create a row index filter
broadcastDvMap.get.value
.get(pathUri)
.map { case DeletionVectorDescriptorWithFilterType(dvDescriptor, filterType) =>
filterType match {
case i if i == RowIndexFilterType.IF_CONTAINED =>
DropMarkedRowsFilter.createInstance(
dvDescriptor,
broadcastHadoopConf.get.value.value,
tablePath.map(new Path(_)))
case i if i == RowIndexFilterType.IF_NOT_CONTAINED =>
KeepMarkedRowsFilter.createInstance(
dvDescriptor,
broadcastHadoopConf.get.value.value,
tablePath.map(new Path(_)))
}
}
.getOrElse(KeepAllRowsFilter)
}
val metadataColumns = Seq(isRowDeletedColumn, rowIndexColumn).filter(_.nonEmpty).map(_.get)
// Unfortunately there is no way to verify the Parquet index is starting from 0.
// We disable the splits, so the assumption is ParquetFileFormat respects that
var rowIndex: Long = 0
// Used only when non-column row batches are received from the Parquet reader
val tempVector = new OnHeapColumnVector(1, ByteType)
iterator.map { row =>
row match {
case batch: ColumnarBatch => // When vectorized Parquet reader is enabled
val size = batch.numRows()
// Create vectors for all needed metadata columns.
// We can't use the one from Parquet reader as it set the
// [[WritableColumnVector.isAllNulls]] to true and it can't be reset with using any
// public APIs.
trySafely(useOffHeapBuffers, size, metadataColumns) { writableVectors =>
val indexVectorTuples = new ArrayBuffer[(Int, ColumnVector)]
var index = 0
isRowDeletedColumn.foreach { columnMetadata =>
val isRowDeletedVector = writableVectors(index)
rowIndexFilter.get
.materializeIntoVector(rowIndex, rowIndex + size, isRowDeletedVector)
indexVectorTuples += (columnMetadata.index -> isRowDeletedVector)
index += 1
}
rowIndexColumn.foreach { columnMetadata =>
val rowIndexVector = writableVectors(index)
// populate the row index column value
for (i <- 0 until size) {
rowIndexVector.putLong(i, rowIndex + i)
}
indexVectorTuples += (columnMetadata.index -> rowIndexVector)
index += 1
}
val newBatch = replaceVectors(batch, indexVectorTuples.toSeq: _*)
rowIndex += size
newBatch
}
case columnarRow: ColumnarBatchRow =>
// When vectorized reader is enabled but returns immutable rows instead of
// columnar batches [[ColumnarBatchRow]]. So we have to copy the row as a
// mutable [[InternalRow]] and set the `row_index` and `is_row_deleted`
// column values. This is not efficient. It should affect only the wide
// tables. https://github.com/delta-io/delta/issues/2246
val newRow = columnarRow.copy();
isRowDeletedColumn.foreach { columnMetadata =>
rowIndexFilter.get.materializeIntoVector(rowIndex, rowIndex + 1, tempVector)
newRow.setByte(columnMetadata.index, tempVector.getByte(0))
}
rowIndexColumn.foreach(columnMetadata => newRow.setLong(columnMetadata.index, rowIndex))
rowIndex += 1
newRow
case rest: InternalRow => // When vectorized Parquet reader is disabled
// Temporary vector variable used to get DV values from RowIndexFilter
// Currently the RowIndexFilter only supports writing into a columnar vector
// and doesn't have methods to get DV value for a specific row index.
// TODO: This is not efficient, but it is ok given the default reader is vectorized
isRowDeletedColumn.foreach { columnMetadata =>
rowIndexFilter.get.materializeIntoVector(rowIndex, rowIndex + 1, tempVector)
rest.setByte(columnMetadata.index, tempVector.getByte(0))
}
rowIndexColumn.foreach(columnMetadata => rest.setLong(columnMetadata.index, rowIndex))
rowIndex += 1
rest
case others =>
throw new RuntimeException(
s"Parquet reader returned an unknown row type: ${others.getClass.getName}")
}
}
}
}
object DeltaParquetFileFormat {
/**
* Column name used to identify whether the row read from the parquet file is marked
* as deleted according to the Delta table deletion vectors
*/
val IS_ROW_DELETED_COLUMN_NAME = "__delta_internal_is_row_deleted"
val IS_ROW_DELETED_STRUCT_FIELD = StructField(IS_ROW_DELETED_COLUMN_NAME, ByteType)
/** Row index for each column */
val ROW_INDEX_COLUMN_NAME = "__delta_internal_row_index"
val ROW_INDEX_STRUCT_FIELD = StructField(ROW_INDEX_COLUMN_NAME, LongType)
/** Utility method to create a new writable vector */
private def newVector(
useOffHeapBuffers: Boolean, size: Int, dataType: StructField): WritableColumnVector = {
if (useOffHeapBuffers) {
OffHeapColumnVector.allocateColumns(size, Seq(dataType).toArray)(0)
} else {
OnHeapColumnVector.allocateColumns(size, Seq(dataType).toArray)(0)
}
}
/** Try the operation, if the operation fails release the created resource */
private def trySafely[R <: WritableColumnVector, T](
useOffHeapBuffers: Boolean,
size: Int,
columns: Seq[ColumnMetadata])(f: Seq[WritableColumnVector] => T): T = {
val resources = new ArrayBuffer[WritableColumnVector](columns.size)
try {
columns.foreach(col => resources.append(newVector(useOffHeapBuffers, size, col.structField)))
f(resources.toSeq)
} catch {
case NonFatal(e) =>
resources.foreach(closeQuietly(_))
throw e
}
}
/** Utility method to quietly close an [[AutoCloseable]] */
private def closeQuietly(closeable: AutoCloseable): Unit = {
if (closeable != null) {
try {
closeable.close()
} catch {
case NonFatal(_) => // ignore
}
}
}
/**
* Helper method to replace the vectors in given [[ColumnarBatch]].
* New vectors and its index in the batch are given as tuples.
*/
private def replaceVectors(
batch: ColumnarBatch,
indexVectorTuples: (Int, ColumnVector) *): ColumnarBatch = {
val vectors = ArrayBuffer[ColumnVector]()
for (i <- 0 until batch.numCols()) {
var replaced: Boolean = false
for (indexVectorTuple <- indexVectorTuples) {
val index = indexVectorTuple._1
val vector = indexVectorTuple._2
if (indexVectorTuple._1 == i) {
vectors += indexVectorTuple._2
// Make sure to close the existing vector allocated in the Parquet
batch.column(i).close()
replaced = true
}
}
if (!replaced) {
vectors += batch.column(i)
}
}
new ColumnarBatch(vectors.toArray, batch.numRows())
}
/** Helper class to encapsulate column info */
case class ColumnMetadata(index: Int, structField: StructField)
/** Helper class that encapsulate an [[RowIndexFilterType]]. */
case class DeletionVectorDescriptorWithFilterType(
descriptor: DeletionVectorDescriptor,
filterType: RowIndexFilterType) {
}
}