Skip to content

Commit cd95799

Browse files
authored
[Spark] Drop Type widening feature: read Parquet footers to collect files to rewrite (#3155)
## What changes were proposed in this pull request? The initial approach to identify files that contain a type that differs from the table schema and that must be rewritten before dropping the type widening table feature is convoluted and turns out to be more brittle than intended. This change switches instead to directly reading the file schema from the Parquet footer and rewriting all files that have a mismatching type. ### Additional Context Files are identified using their default row commit version (a part of the row tracking feature) and matched against type changes previously applied to the table and recorded in the table metadata: any file written before the latest type change should use a different type and must be rewritten. This requires multiple pieces of information to be accurately tracked: - Default row commit versions must be correctly assigned to all files. E.p. files that are copied over without modification must never be assigned a new default row commit version. On the other hand, default row commit versions are preserved across CLONE but these versions don't match anything in the new cloned table. - Type change history must be reliably recorded and preserved across schema changes, e.g. column mapping. Any bug will likely lead to files not being correctly rewritten before removing the table feature, potentially leaving the table in an unreadable state. ## How was this patch tested? Tests added in previous PR to cover CLONE and RESTORE: #3053 Tests added and updated in this PR to cover rewriting files with different column types when removing the table feature.
1 parent 4b102d3 commit cd95799

File tree

6 files changed

+174
-45
lines changed

6 files changed

+174
-45
lines changed

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

Lines changed: 7 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,9 +23,9 @@ import scala.util.control.NonFatal
2323
import org.apache.spark.sql.delta.catalog.DeltaTableV2
2424
import org.apache.spark.sql.delta.commands.{AlterTableSetPropertiesDeltaCommand, AlterTableUnsetPropertiesDeltaCommand, DeltaReorgTableCommand, DeltaReorgTableMode, DeltaReorgTableSpec}
2525
import org.apache.spark.sql.delta.commands.columnmapping.RemoveColumnMappingCommand
26+
import org.apache.spark.sql.delta.commands.optimize.OptimizeMetrics
2627
import org.apache.spark.sql.delta.managedcommit.ManagedCommitUtils
2728
import org.apache.spark.sql.delta.metering.DeltaLogging
28-
import org.apache.spark.sql.delta.schema.SchemaMergingUtils
2929
import org.apache.spark.sql.delta.util.{Utils => DeltaUtils}
3030
import org.apache.spark.sql.util.ScalaExtensions._
3131

@@ -309,8 +309,9 @@ case class TypeWideningPreDowngradeCommand(table: DeltaTableV2)
309309
* @return Return the number of files rewritten.
310310
*/
311311
private def rewriteFilesIfNeeded(): Long = {
312-
val numFilesToRewrite = TypeWidening.numFilesRequiringRewrite(table.initialSnapshot)
313-
if (numFilesToRewrite == 0L) return 0L
312+
if (!TypeWideningMetadata.containsTypeWideningMetadata(table.initialSnapshot.schema)) {
313+
return 0L
314+
}
314315

315316
// Wrap `table` in a ResolvedTable that can be passed to DeltaReorgTableCommand. The catalog &
316317
// table ID won't be used by DeltaReorgTableCommand.
@@ -323,8 +324,9 @@ case class TypeWideningPreDowngradeCommand(table: DeltaTableV2)
323324
reorgTableSpec = DeltaReorgTableSpec(DeltaReorgTableMode.REWRITE_TYPE_WIDENING, None)
324325
)(Nil)
325326

326-
reorg.run(table.spark)
327-
numFilesToRewrite
327+
val rows = reorg.run(table.spark)
328+
val metrics = rows.head.getAs[OptimizeMetrics](1)
329+
metrics.numFilesRemoved
328330
}
329331

330332
/**

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

Lines changed: 0 additions & 29 deletions
Original file line numberDiff line numberDiff line change
@@ -96,33 +96,4 @@ object TypeWidening {
9696
)
9797
}
9898
}
99-
100-
/**
101-
* Filter the given list of files to only keep files that were written before the latest type
102-
* change, if any. These older files contain a column or field with a type that is different than
103-
* in the current table schema and must be rewritten when dropping the type widening table feature
104-
* to make the table readable by readers that don't support the feature.
105-
*/
106-
def filterFilesRequiringRewrite(snapshot: Snapshot, files: Seq[AddFile]): Seq[AddFile] =
107-
TypeWideningMetadata.getLatestTypeChangeVersion(snapshot.metadata.schema) match {
108-
case Some(latestVersion) =>
109-
files.filter(_.defaultRowCommitVersion match {
110-
case Some(version) => version < latestVersion
111-
// Files written before the type widening table feature was added to the table don't
112-
// have a defaultRowCommitVersion. That does mean they were written before the latest
113-
// type change.
114-
case None => true
115-
})
116-
case None =>
117-
Seq.empty
118-
}
119-
120-
121-
/**
122-
* Return the number of files that were written before the latest type change and that then
123-
* contain a column or field with a type that is different from the current able schema.
124-
*/
125-
def numFilesRequiringRewrite(snapshot: Snapshot): Long = {
126-
filterFilesRequiringRewrite(snapshot, snapshot.allFiles.collect()).size
127-
}
12899
}

spark/src/main/scala/org/apache/spark/sql/delta/commands/DeltaReorgTableCommand.scala

Lines changed: 14 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -16,7 +16,7 @@
1616

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

19-
import org.apache.spark.sql.delta.{Snapshot, TypeWidening}
19+
import org.apache.spark.sql.delta.{DeltaColumnMapping, Snapshot}
2020
import org.apache.spark.sql.delta.actions.AddFile
2121

2222
import org.apache.spark.sql.{Row, SparkSession}
@@ -97,14 +97,15 @@ sealed trait DeltaReorgOperation {
9797
* Collects files that need to be processed by the reorg operation from the list of candidate
9898
* files.
9999
*/
100-
def filterFilesToReorg(snapshot: Snapshot, files: Seq[AddFile]): Seq[AddFile]
100+
def filterFilesToReorg(spark: SparkSession, snapshot: Snapshot, files: Seq[AddFile]): Seq[AddFile]
101101
}
102102

103103
/**
104104
* Reorg operation to purge files with soft deleted rows.
105105
*/
106106
class DeltaPurgeOperation extends DeltaReorgOperation {
107-
override def filterFilesToReorg(snapshot: Snapshot, files: Seq[AddFile]): Seq[AddFile] =
107+
override def filterFilesToReorg(spark: SparkSession, snapshot: Snapshot, files: Seq[AddFile])
108+
: Seq[AddFile] =
108109
files.filter { file =>
109110
(file.deletionVector != null && file.numPhysicalRecords.isEmpty) ||
110111
file.numDeletedRecords > 0L
@@ -115,7 +116,8 @@ class DeltaPurgeOperation extends DeltaReorgOperation {
115116
* Reorg operation to upgrade the iceberg compatibility version of a table.
116117
*/
117118
class DeltaUpgradeUniformOperation(icebergCompatVersion: Int) extends DeltaReorgOperation {
118-
override def filterFilesToReorg(snapshot: Snapshot, files: Seq[AddFile]): Seq[AddFile] = {
119+
override def filterFilesToReorg(spark: SparkSession, snapshot: Snapshot, files: Seq[AddFile])
120+
: Seq[AddFile] = {
119121
def shouldRewriteToBeIcebergCompatible(file: AddFile): Boolean = {
120122
if (file.tags == null) return true
121123
val icebergCompatVersion = file.tags.getOrElse(AddFile.Tags.ICEBERG_COMPAT_VERSION.name, "0")
@@ -129,7 +131,12 @@ class DeltaUpgradeUniformOperation(icebergCompatVersion: Int) extends DeltaReorg
129131
* Internal reorg operation to rewrite files to conform to the current table schema when dropping
130132
* the type widening table feature.
131133
*/
132-
class DeltaRewriteTypeWideningOperation extends DeltaReorgOperation {
133-
override def filterFilesToReorg(snapshot: Snapshot, files: Seq[AddFile]): Seq[AddFile] =
134-
TypeWidening.filterFilesRequiringRewrite(snapshot, files)
134+
class DeltaRewriteTypeWideningOperation extends DeltaReorgOperation with ReorgTableHelper {
135+
override def filterFilesToReorg(spark: SparkSession, snapshot: Snapshot, files: Seq[AddFile])
136+
: Seq[AddFile] = {
137+
val physicalSchema = DeltaColumnMapping.renameColumns(snapshot.schema)
138+
filterParquetFilesOnExecutors(spark, files, snapshot, ignoreCorruptFiles = false) {
139+
schema => fileHasDifferentTypes(schema, physicalSchema)
140+
}
141+
}
135142
}

spark/src/main/scala/org/apache/spark/sql/delta/commands/OptimizeTableCommand.scala

Lines changed: 4 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -265,8 +265,10 @@ class OptimizeExecutor(
265265
val partitionSchema = txn.metadata.partitionSchema
266266

267267
val filesToProcess = optimizeContext.reorg match {
268-
case Some(reorgOperation) => reorgOperation.filterFilesToReorg(txn.snapshot, candidateFiles)
269-
case None => filterCandidateFileList(minFileSize, maxDeletedRowsRatio, candidateFiles)
268+
case Some(reorgOperation) =>
269+
reorgOperation.filterFilesToReorg(sparkSession, txn.snapshot, candidateFiles)
270+
case None =>
271+
filterCandidateFileList(minFileSize, maxDeletedRowsRatio, candidateFiles)
270272
}
271273
val partitionsToCompact = filesToProcess.groupBy(_.partitionValues).toSeq
272274

Lines changed: 111 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,111 @@
1+
/*
2+
* Copyright (2021) The Delta Lake Project Authors.
3+
*
4+
* Licensed under the Apache License, Version 2.0 (the "License");
5+
* you may not use this file except in compliance with the License.
6+
* You may obtain a copy of the License at
7+
*
8+
* http://www.apache.org/licenses/LICENSE-2.0
9+
*
10+
* Unless required by applicable law or agreed to in writing, software
11+
* distributed under the License is distributed on an "AS IS" BASIS,
12+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13+
* See the License for the specific language governing permissions and
14+
* limitations under the License.
15+
*/
16+
17+
package org.apache.spark.sql.delta.commands
18+
19+
import org.apache.spark.sql.delta.Snapshot
20+
import org.apache.spark.sql.delta.actions.AddFile
21+
import org.apache.spark.sql.delta.commands.VacuumCommand.generateCandidateFileMap
22+
import org.apache.spark.sql.delta.schema.SchemaMergingUtils
23+
import org.apache.spark.sql.delta.util.DeltaFileOperations
24+
import org.apache.hadoop.conf.Configuration
25+
import org.apache.hadoop.fs.{FileStatus, Path}
26+
27+
import org.apache.spark.sql.SparkSession
28+
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetToSparkSchemaConverter}
29+
import org.apache.spark.sql.types.{AtomicType, StructField, StructType}
30+
import org.apache.spark.util.SerializableConfiguration
31+
32+
trait ReorgTableHelper extends Serializable {
33+
/**
34+
* Determine whether `fileSchema` has any columns that has a type that differs from
35+
* `tablePhysicalSchema`.
36+
*/
37+
protected def fileHasDifferentTypes(
38+
fileSchema: StructType,
39+
tablePhysicalSchema: StructType): Boolean = {
40+
SchemaMergingUtils.transformColumns(fileSchema, tablePhysicalSchema) {
41+
case (_, StructField(_, fileType: AtomicType, _, _),
42+
Some(StructField(_, tableType: AtomicType, _, _)), _) if fileType != tableType =>
43+
return true
44+
case (_, field, _, _) => field
45+
}
46+
false
47+
}
48+
49+
/**
50+
* Apply a filter on the list of AddFile to only keep the files that have physical parquet schema
51+
* that satisfies the given filter function.
52+
*
53+
* Note: Filtering happens on the executors: **any variable captured by `filterFileFn` must be
54+
* Serializable**
55+
*/
56+
protected def filterParquetFilesOnExecutors(
57+
spark: SparkSession,
58+
files: Seq[AddFile],
59+
snapshot: Snapshot,
60+
ignoreCorruptFiles: Boolean)(
61+
filterFileFn: StructType => Boolean): Seq[AddFile] = {
62+
63+
val serializedConf = new SerializableConfiguration(snapshot.deltaLog.newDeltaHadoopConf())
64+
val assumeBinaryIsString = spark.sessionState.conf.isParquetBinaryAsString
65+
val assumeInt96IsTimestamp = spark.sessionState.conf.isParquetINT96AsTimestamp
66+
val dataPath = new Path(snapshot.deltaLog.dataPath.toString)
67+
68+
import org.apache.spark.sql.delta.implicits._
69+
70+
files.toDF(spark).as[AddFile].mapPartitions { iter =>
71+
filterParquetFiles(iter.toList, dataPath, serializedConf.value, ignoreCorruptFiles,
72+
assumeBinaryIsString, assumeInt96IsTimestamp)(filterFileFn).toIterator
73+
}.collect()
74+
}
75+
76+
protected def filterParquetFiles(
77+
files: Seq[AddFile],
78+
dataPath: Path,
79+
configuration: Configuration,
80+
ignoreCorruptFiles: Boolean,
81+
assumeBinaryIsString: Boolean,
82+
assumeInt96IsTimestamp: Boolean)(
83+
filterFileFn: StructType => Boolean): Seq[AddFile] = {
84+
val nameToAddFileMap = generateCandidateFileMap(dataPath, files)
85+
86+
val fileStatuses = nameToAddFileMap.map { case (absPath, addFile) =>
87+
new FileStatus(
88+
/* length */ addFile.size,
89+
/* isDir */ false,
90+
/* blockReplication */ 0,
91+
/* blockSize */ 1,
92+
/* modificationTime */ addFile.modificationTime,
93+
new Path(absPath)
94+
)
95+
}
96+
97+
val footers = DeltaFileOperations.readParquetFootersInParallel(
98+
configuration,
99+
fileStatuses.toList,
100+
ignoreCorruptFiles)
101+
102+
val converter =
103+
new ParquetToSparkSchemaConverter(assumeBinaryIsString, assumeInt96IsTimestamp)
104+
105+
val filesNeedToRewrite = footers.filter { footer =>
106+
val fileSchema = ParquetFileFormat.readSchemaFromFooter(footer, converter)
107+
filterFileFn(fileSchema)
108+
}.map(_.getFile.toString)
109+
filesNeedToRewrite.map(absPath => nameToAddFileMap(absPath))
110+
}
111+
}

spark/src/test/scala/org/apache/spark/sql/delta/DeltaTypeWideningSuite.scala

Lines changed: 38 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -16,6 +16,7 @@
1616

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

19+
import java.io.{File, PrintWriter}
1920
import java.util.concurrent.TimeUnit
2021

2122
import com.databricks.spark.util.Log4jUsageLogger
@@ -1017,7 +1018,7 @@ trait DeltaTypeWideningStatsTests {
10171018
* Tests covering adding and removing the type widening table feature. Dropping the table feature
10181019
* also includes rewriting data files with the old type and removing type widening metadata.
10191020
*/
1020-
trait DeltaTypeWideningTableFeatureTests {
1021+
trait DeltaTypeWideningTableFeatureTests extends DeltaTypeWideningTestCases {
10211022
self: QueryTest
10221023
with ParquetTest
10231024
with RowTrackingTestUtils
@@ -1351,6 +1352,21 @@ trait DeltaTypeWideningTableFeatureTests {
13511352
}
13521353
}
13531354

1355+
for {
1356+
testCase <- supportedTestCases
1357+
}
1358+
test(s"drop feature after type change ${testCase.fromType.sql} -> ${testCase.toType.sql}") {
1359+
append(testCase.initialValuesDF.repartition(2))
1360+
sql(s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN value TYPE ${testCase.toType.sql}")
1361+
append(testCase.additionalValuesDF.repartition(3))
1362+
dropTableFeature(
1363+
expectedOutcome = ExpectedOutcome.FAIL_CURRENT_VERSION_USES_FEATURE,
1364+
expectedNumFilesRewritten = 2,
1365+
expectedColumnTypes = Map("value" -> testCase.toType)
1366+
)
1367+
checkAnswer(readDeltaTable(tempPath), testCase.expectedResult)
1368+
}
1369+
13541370
test("drop feature after a type change with schema evolution") {
13551371
setupManualClock()
13561372
sql(s"CREATE TABLE delta.`$tempPath` (a byte) USING DELTA")
@@ -1453,7 +1469,7 @@ trait DeltaTypeWideningTableFeatureTests {
14531469

14541470
dropTableFeature(
14551471
expectedOutcome = ExpectedOutcome.FAIL_CURRENT_VERSION_USES_FEATURE,
1456-
expectedNumFilesRewritten = 3,
1472+
expectedNumFilesRewritten = 2,
14571473
expectedColumnTypes = Map("a" -> IntegerType)
14581474
)
14591475
checkAnswer(readDeltaTable(tempPath),
@@ -1498,6 +1514,26 @@ trait DeltaTypeWideningTableFeatureTests {
14981514
)
14991515
checkAnswer(readDeltaTable(tempPath), Seq(Row(1), Row(2)))
15001516
}
1517+
1518+
test("rewriting files fails if there are corrupted files") {
1519+
sql(s"CREATE TABLE delta.`$tempPath` (a byte) USING DELTA")
1520+
sql(s"ALTER TABLE delta.`$tempPath` CHANGE COLUMN a TYPE INT")
1521+
addSingleFile(Seq(2), IntegerType)
1522+
addSingleFile(Seq(3), IntegerType)
1523+
val filePath = deltaLog.update().allFiles.first().path
1524+
val pw = new PrintWriter(new File(tempPath, filePath))
1525+
pw.write("corrupted")
1526+
pw.close()
1527+
1528+
// Rewriting files when dropping type widening should ignore this config, if the corruption is
1529+
// transient it will leave files behind that some clients can't read.
1530+
withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") {
1531+
val ex = intercept[SparkException] {
1532+
sql(s"ALTER TABLE delta.`$tempDir` DROP FEATURE '${TypeWideningTableFeature.name}'")
1533+
}
1534+
assert(ex.getMessage.contains("Cannot seek after EOF"))
1535+
}
1536+
}
15011537
}
15021538

15031539
/** Trait collecting tests covering type widening + column mapping. */

0 commit comments

Comments
 (0)