Skip to content

Commit 90b98e3

Browse files
authored
Automatic type widening in MERGE (#2764)
<!-- 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 This change is part of the type widening table feature. Type widening feature request: #2622 Type Widening protocol RFC: #2624 It adds automatic type widening as part of schema evolution in MERGE INTO: - During resolution of the `DeltaMergeInto` plan, when merging the target and source schema to compute the schema after evolution, we keep the wider source type when type widening is enabled on the table. - When updating the table schema at the beginning of MERGE execution, metadata is added to the schema to record type changes. ## How was this patch tested? - A new test suite `DeltaTypeWideningSchemaEvolutionSuite` is added to cover type evolution in MERGE ## This PR introduces the following *user-facing* changes The table feature is available in testing only, there are no user-facing changes as of now. When automatic schema evolution is enabled in MERGE and the source schema contains a type that is wider than the target schema: With type widening disabled: the type in the target schema is not changed. the ingestion behavior follows the `storeAssignmentPolicy` configuration: - LEGACY: source values that overflow the target type are stored as `null` - ANSI: a runtime check is injected to fail on source values that overflow the target type. - STRICT: the MERGE operation fails during analysis. With type widening enabled: the type in the target schema is updated to the wider source type. The MERGE operation always succeeds: ``` -- target: key int, value short -- source: key int, value int MERGE INTO target USING source ON target.key = source.key WHEN MATCHED THEN UPDATE SET * ``` After the MERGE operation, the target schema is `key int, value int`.
1 parent cb07092 commit 90b98e3

11 files changed

+507
-67
lines changed

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

+10-2
Original file line numberDiff line numberDiff line change
@@ -234,7 +234,7 @@ object ResolveDeltaMergeInto {
234234
// schema before merging it with the target schema. We don't consider NOT MATCHED BY SOURCE
235235
// clauses since these can't by definition reference source columns and thus can't introduce
236236
// new columns in the target schema.
237-
val actions = (matchedClauses ++ notMatchedClauses).flatMap(_.actions)
237+
val actions = (resolvedMatchedClauses ++ resolvedNotMatchedClauses).flatMap(_.actions)
238238
val assignments = actions.collect { case a: DeltaMergeAction => a.targetColNameParts }
239239
val containsStarAction = actions.exists {
240240
case _: UnresolvedStar => true
@@ -278,14 +278,22 @@ object ResolveDeltaMergeInto {
278278
})
279279

280280
val migrationSchema = filterSchema(source.schema, Seq.empty)
281+
val allowTypeWidening = target.exists {
282+
case DeltaTable(fileIndex) =>
283+
TypeWidening.isEnabled(fileIndex.protocol, fileIndex.metadata)
284+
case _ => false
285+
}
286+
281287
// The implicit conversions flag allows any type to be merged from source to target if Spark
282288
// SQL considers the source type implicitly castable to the target. Normally, mergeSchemas
283289
// enforces Parquet-level write compatibility, which would mean an INT source can't be merged
284290
// into a LONG target.
285291
SchemaMergingUtils.mergeSchemas(
286292
target.schema,
287293
migrationSchema,
288-
allowImplicitConversions = true)
294+
allowImplicitConversions = true,
295+
allowTypeWidening = allowTypeWidening
296+
)
289297
} else {
290298
target.schema
291299
}

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

+27
Original file line numberDiff line numberDiff line change
@@ -47,6 +47,20 @@ object TypeWidening {
4747
isEnabled
4848
}
4949

50+
/**
51+
* Checks that the type widening table property wasn't disabled or enabled between the two given
52+
* states, throws an errors if it was.
53+
*/
54+
def ensureFeatureConsistentlyEnabled(
55+
protocol: Protocol,
56+
metadata: Metadata,
57+
otherProtocol: Protocol,
58+
otherMetadata: Metadata): Unit = {
59+
if (isEnabled(protocol, metadata) != isEnabled(otherProtocol, otherMetadata)) {
60+
throw DeltaErrors.metadataChangedException(None)
61+
}
62+
}
63+
5064
/**
5165
* Returns whether the given type change is eligible for widening. This only checks atomic types.
5266
* It is the responsibility of the caller to recurse into structs, maps and arrays.
@@ -62,6 +76,19 @@ object TypeWidening {
6276
case _ => false
6377
}
6478

79+
/**
80+
* Returns whether the given type change can be applied during schema evolution. Only a
81+
* subset of supported type changes are considered for schema evolution.
82+
*/
83+
def isTypeChangeSupportedForSchemaEvolution(fromType: AtomicType, toType: AtomicType): Boolean =
84+
(fromType, toType) match {
85+
case (from, to) if from == to => true
86+
case (from, to) if !isTypeChangeSupported(from, to) => false
87+
case (ByteType, ShortType) => true
88+
case (ByteType | ShortType, IntegerType) => true
89+
case _ => false
90+
}
91+
6592
/**
6693
* Filter the given list of files to only keep files that were written before the latest type
6794
* change, if any. These older files contain a column or field with a type that is different than

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

+9
Original file line numberDiff line numberDiff line change
@@ -91,6 +91,15 @@ case class MergeIntoCommand(
9191
atAnalysis = target.schema, latestSchema = deltaTxn.metadata.schema)
9292
}
9393

94+
// Check that type widening wasn't enabled/disabled between analysis and the start of the
95+
// transaction.
96+
TypeWidening.ensureFeatureConsistentlyEnabled(
97+
protocol = targetFileIndex.protocol,
98+
metadata = targetFileIndex.metadata,
99+
otherProtocol = deltaTxn.protocol,
100+
otherMetadata = deltaTxn.metadata
101+
)
102+
94103
if (canMergeSchema) {
95104
updateMetadata(
96105
spark, deltaTxn, migratedSchema.getOrElse(target.schema),

spark/src/main/scala/org/apache/spark/sql/delta/schema/ImplicitMetadataOperation.scala

+10-2
Original file line numberDiff line numberDiff line change
@@ -122,7 +122,14 @@ trait ImplicitMetadataOperation extends DeltaLogging {
122122
if (rearrangeOnly) {
123123
throw DeltaErrors.unexpectedDataChangeException("Change the Delta table schema")
124124
}
125-
txn.updateMetadata(txn.metadata.copy(schemaString = mergedSchema.json
125+
126+
val schemaWithTypeWideningMetadata = TypeWideningMetadata.addTypeWideningMetadata(
127+
txn,
128+
schema = mergedSchema,
129+
oldSchema = txn.metadata.schema
130+
)
131+
132+
txn.updateMetadata(txn.metadata.copy(schemaString = schemaWithTypeWideningMetadata.json
126133
))
127134
} else if (isNewSchema || isNewPartitioning
128135
) {
@@ -201,7 +208,8 @@ object ImplicitMetadataOperation {
201208
SchemaMergingUtils.mergeSchemas(
202209
txn.metadata.schema,
203210
dataSchema,
204-
fixedTypeColumns = fixedTypeColumns)
211+
fixedTypeColumns = fixedTypeColumns,
212+
allowTypeWidening = TypeWidening.isEnabled(txn.protocol, txn.metadata))
205213
}
206214
}
207215

spark/src/main/scala/org/apache/spark/sql/delta/schema/SchemaMergingUtils.scala

+6-3
Original file line numberDiff line numberDiff line change
@@ -20,14 +20,13 @@ import java.util.Locale
2020

2121
import scala.util.control.NonFatal
2222

23-
import org.apache.spark.sql.delta.DeltaAnalysisException
23+
import org.apache.spark.sql.delta.{DeltaAnalysisException, TypeWidening}
2424

2525
import org.apache.spark.sql.AnalysisException
2626
import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion, UnresolvedAttribute}
2727
import org.apache.spark.sql.catalyst.expressions.Literal
28-
import org.apache.spark.sql.catalyst.plans.logical.DeltaMergeInto
2928
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
30-
import org.apache.spark.sql.types.{ArrayType, ByteType, DataType, DecimalType, IntegerType, MapType, NullType, ShortType, StructField, StructType}
29+
import org.apache.spark.sql.types._
3130

3231
/**
3332
* Utils to merge table schema with data schema.
@@ -168,6 +167,7 @@ object SchemaMergingUtils {
168167
dataSchema: StructType,
169168
allowImplicitConversions: Boolean = false,
170169
keepExistingType: Boolean = false,
170+
allowTypeWidening: Boolean = false,
171171
fixedTypeColumns: Set[String] = Set.empty,
172172
caseSensitive: Boolean = false): StructType = {
173173
checkColumnNameDuplication(dataSchema, "in the data to save", caseSensitive)
@@ -232,6 +232,9 @@ object SchemaMergingUtils {
232232
// Simply keeps the existing type for primitive types
233233
case (current, update) if keepExistingType => current
234234

235+
case (current: AtomicType, update: AtomicType) if allowTypeWidening &&
236+
TypeWidening.isTypeChangeSupportedForSchemaEvolution(current, update) => update
237+
235238
// If implicit conversions are allowed, that means we can use any valid implicit cast to
236239
// perform the merge.
237240
case (current, update)

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

+6-1
Original file line numberDiff line numberDiff line change
@@ -1432,7 +1432,12 @@ trait DeltaErrorsSuiteBase
14321432
val e = intercept[DeltaAnalysisException] {
14331433
val s1 = StructType(Seq(StructField("c0", IntegerType, true)))
14341434
val s2 = StructType(Seq(StructField("c0", StringType, false)))
1435-
SchemaMergingUtils.mergeSchemas(s1, s2, false, false, Set("c0"))
1435+
SchemaMergingUtils.mergeSchemas(s1, s2,
1436+
allowImplicitConversions = false,
1437+
keepExistingType = false,
1438+
allowTypeWidening = false,
1439+
Set("c0")
1440+
)
14361441
}
14371442
checkErrorMessage(e, Some("DELTA_GENERATED_COLUMNS_DATA_TYPE_MISMATCH"), Some("42K09"),
14381443
Some("Column c0 is a generated column or a column used by a generated " +

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

+21-1
Original file line numberDiff line numberDiff line change
@@ -42,10 +42,11 @@ import org.apache.spark.scheduler.{JobFailed, SparkListener, SparkListenerJobEnd
4242
import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession}
4343
import org.apache.spark.sql.catalyst.TableIdentifier
4444
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
45-
import org.apache.spark.sql.catalyst.util.quietly
45+
import org.apache.spark.sql.catalyst.util.{quietly, FailFastMode}
4646
import org.apache.spark.sql.execution.{FileSourceScanExec, QueryExecution, RDDScanExec, SparkPlan, WholeStageCodegenExec}
4747
import org.apache.spark.sql.execution.aggregate.HashAggregateExec
4848
import org.apache.spark.sql.test.SharedSparkSession
49+
import org.apache.spark.sql.types.StructType
4950
import org.apache.spark.sql.util.QueryExecutionListener
5051
import org.apache.spark.util.Utils
5152

@@ -475,6 +476,8 @@ trait DeltaDMLTestUtils
475476
with BeforeAndAfterEach {
476477
self: SharedSparkSession =>
477478

479+
import testImplicits._
480+
478481
protected var tempDir: File = _
479482

480483
protected var deltaLog: DeltaLog = _
@@ -523,6 +526,23 @@ trait DeltaDMLTestUtils
523526
}
524527
}
525528

529+
/**
530+
* Parse the input JSON data into a dataframe, one row per input element.
531+
* Throws an exception on malformed inputs or records that don't comply with the provided schema.
532+
*/
533+
protected def readFromJSON(data: Seq[String], schema: StructType = null): DataFrame = {
534+
if (schema != null) {
535+
spark.read
536+
.schema(schema)
537+
.option("mode", FailFastMode.name)
538+
.json(data.toDS)
539+
} else {
540+
spark.read
541+
.option("mode", FailFastMode.name)
542+
.json(data.toDS)
543+
}
544+
}
545+
526546
protected def readDeltaTable(path: String): DataFrame = {
527547
spark.read.format("delta").load(path)
528548
}

0 commit comments

Comments
 (0)