Skip to content

Commit d59e3f0

Browse files
authored
[Spark] Automatic type widening in Delta streaming sink (#3626)
## Description This change introduces automatic type widening during schema evolution in the Delta streaming sink. Conditions for type widening to trigger: - Type widening is enabled on the Delta table - Schema evolution (`mergeSchema`) is enabled on the sink - The data written to the sink uses a type that is strictly wider than the current type in the table schema, and moving from the narrower to the wider type is eligible for type widening - see `TypeWidening.isTypeChangeSupportedForSchemaEvolution` When all conditions are satisfied, the table schema is updated to use the wider type before ingesting the data. ## How was this patch tested? Added test suite `TypeWideningStreamingSinkSuite` covering type widening in the Delta streaming sink ## Does this PR introduce _any_ user-facing changes? This builds on the user-facing change introduced in #3443 that allows writing to a delta sink using a different type than the current table type. Without type widening: ``` spark.readStream .table("delta_source") # Column 'a' has type INT in 'delta_sink'. .select(col("a").cast("long").alias("a")) .writeStream .format("delta") .option("checkpointLocation", "<location>") .toTable("delta_sink") ``` The write to the sink succeeds, column `a` retains its type `INT` and the data is cast from `LONG` to `INT` on write. With type widening: ``` spark.sql("ALTER TABLE delta_sink SET TBLPROPERTIES ('delta.enableTypeWidening' = 'true')") spark.readStream .table("delta_source") # Column 'a' has type INT in 'delta_sink'. .select(col("a").cast("long").alias("a")) .writeStream .format("delta") .option("checkpointLocation", "<location>") .option("mergeSchema", "true") .toTable("delta_sink") ``` The write to sink succeeds, the type of column `a` is changed from `INT` to `LONG`, data is ingested as `LONG`.
1 parent 9bd2c43 commit d59e3f0

File tree

3 files changed

+250
-14
lines changed

3 files changed

+250
-14
lines changed

spark/src/main/scala/org/apache/spark/sql/delta/sources/DeltaSink.scala

Lines changed: 11 additions & 9 deletions
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,7 @@ import java.util.concurrent.ConcurrentHashMap
2020

2121
import org.apache.spark.sql.delta._
2222
import org.apache.spark.sql.delta.DeltaOperations.StreamingUpdate
23-
import org.apache.spark.sql.delta.actions.{FileAction, Metadata, SetTransaction}
23+
import org.apache.spark.sql.delta.actions.{FileAction, Metadata, Protocol, SetTransaction}
2424
import org.apache.spark.sql.delta.logging.DeltaLogKeys
2525
import org.apache.spark.sql.delta.metering.DeltaLogging
2626
import org.apache.spark.sql.delta.schema.{ImplicitMetadataOperation, SchemaMergingUtils, SchemaUtils}
@@ -29,9 +29,8 @@ import org.apache.hadoop.fs.Path
2929
// scalastyle:off import.ordering.noEmptyLine
3030
import org.apache.spark.internal.MDC
3131
import org.apache.spark.sql._
32-
import org.apache.spark.sql.catalyst.analysis.TableOutputResolver
3332
import org.apache.spark.sql.catalyst.catalog.CatalogTable
34-
import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, Expression}
33+
import org.apache.spark.sql.catalyst.expressions.Alias
3534
import org.apache.spark.sql.catalyst.types.DataTypeUtils
3635
import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
3736
import org.apache.spark.sql.execution.{QueryExecution, SQLExecution}
@@ -130,7 +129,7 @@ case class DeltaSink(
130129
txn.readWholeTable()
131130
}
132131

133-
val writeSchema = getWriteSchema(txn.metadata, data.schema)
132+
val writeSchema = getWriteSchema(txn.protocol, txn.metadata, data.schema)
134133
// Streaming sinks can't blindly overwrite schema. See Schema Management design doc for details
135134
updateMetadata(data.sparkSession, txn, writeSchema, partitionColumns, Map.empty,
136135
outputMode == OutputMode.Complete(), rearrangeOnly = false)
@@ -168,17 +167,20 @@ case class DeltaSink(
168167

169168
/**
170169
* Returns the schema to use to write data to this delta table. The write schema includes new
171-
* columns to add with schema evolution and reconciles types to match the table types.
170+
* columns to add with schema evolution and reconciles types to match the table types when
171+
* possible or apply type widening if enabled.
172172
*/
173-
private def getWriteSchema(metadata: Metadata, dataSchema: StructType): StructType = {
173+
private def getWriteSchema(
174+
protocol: Protocol, metadata: Metadata, dataSchema: StructType): StructType = {
174175
if (!sqlConf.getConf(DeltaSQLConf.DELTA_STREAMING_SINK_ALLOW_IMPLICIT_CASTS)) return dataSchema
175176

176177
if (canOverwriteSchema) return dataSchema
177178

178179
SchemaMergingUtils.mergeSchemas(
179-
tableSchema = metadata.schema,
180-
dataSchema = dataSchema,
181-
allowImplicitConversions = true
180+
metadata.schema,
181+
dataSchema,
182+
allowImplicitConversions = true,
183+
allowTypeWidening = canMergeSchema && TypeWidening.isEnabled(protocol, metadata)
182184
)
183185
}
184186

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

Lines changed: 10 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -28,13 +28,14 @@ import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamExecution}
2828
import org.apache.spark.sql.functions.{col, lit}
2929
import org.apache.spark.sql.internal.SQLConf
3030
import org.apache.spark.sql.internal.SQLConf.StoreAssignmentPolicy
31-
import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException}
31+
import org.apache.spark.sql.streaming.{OutputMode, StreamingQueryException, Trigger}
3232
import org.apache.spark.sql.types._
3333

3434
/**
35-
* Covers handling implicit casting to handle type mismatches when writing data to a Delta sink.
35+
* Defines helper class & methods to test writing to a Delta streaming sink using data types that
36+
* don't match the corresponding column type in the table schema.
3637
*/
37-
abstract class DeltaSinkImplicitCastTest extends DeltaSinkTest {
38+
abstract class DeltaSinkImplicitCastSuiteBase extends DeltaSinkTest {
3839

3940
override def beforeAll(): Unit = {
4041
super.beforeAll()
@@ -59,6 +60,7 @@ abstract class DeltaSinkImplicitCastTest extends DeltaSinkTest {
5960
extraOptions: Map[String, String])(
6061
data: T*)(
6162
selectExpr: String*): Unit = {
63+
source.addData(data)
6264
val query =
6365
source.toDF()
6466
.selectExpr(selectExpr: _*)
@@ -67,9 +69,9 @@ abstract class DeltaSinkImplicitCastTest extends DeltaSinkTest {
6769
.outputMode(outputMode)
6870
.options(extraOptions)
6971
.format("delta")
72+
.trigger(Trigger.AvailableNow())
7073
.start(outputDir.getCanonicalPath)
7174
try {
72-
source.addData(data)
7375
failAfter(streamingTimeout) {
7476
query.processAllAvailable()
7577
}
@@ -106,7 +108,10 @@ abstract class DeltaSinkImplicitCastTest extends DeltaSinkTest {
106108
}
107109
}
108110

109-
class DeltaSinkImplicitCastSuite extends DeltaSinkImplicitCastTest {
111+
/**
112+
* Covers handling implicit casting to handle type mismatches when writing data to a Delta sink.
113+
*/
114+
class DeltaSinkImplicitCastSuite extends DeltaSinkImplicitCastSuiteBase {
110115
import testImplicits._
111116

112117
test(s"write wider type - long -> int") {
Lines changed: 229 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,229 @@
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.typewidening
18+
19+
import org.apache.spark.sql.delta._
20+
import org.apache.spark.sql.delta.sources.{DeltaSink, DeltaSQLConf}
21+
22+
import org.apache.spark.sql.Row
23+
import org.apache.spark.sql.execution.streaming.StreamExecution
24+
import org.apache.spark.sql.internal.SQLConf
25+
import org.apache.spark.sql.streaming.OutputMode
26+
import org.apache.spark.sql.types._
27+
28+
/**
29+
* Suite covering automatic type widening in the Delta streaming sink.
30+
*/
31+
class TypeWideningStreamingSinkSuite
32+
extends DeltaSinkImplicitCastSuiteBase
33+
with TypeWideningTestMixin {
34+
35+
import testImplicits._
36+
37+
override def beforeAll(): Unit = {
38+
super.beforeAll()
39+
// Set by default confs to enable automatic type widening in all tests. Negative tests should
40+
// explicitly disable these.
41+
spark.conf.set(DeltaSQLConf.DELTA_STREAMING_SINK_ALLOW_IMPLICIT_CASTS.key, "true")
42+
spark.conf.set(DeltaConfigs.ENABLE_TYPE_WIDENING.defaultTablePropertyKey, "true")
43+
spark.conf.set(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key, "true")
44+
// Ensure we don't silently cast test inputs to null on overflow.
45+
spark.conf.set(SQLConf.ANSI_ENABLED.key, "true")
46+
}
47+
48+
test("type isn't widened if schema evolution is disabled") {
49+
withDeltaStream[Int] { stream =>
50+
stream.write(17)("CAST(value AS SHORT)")
51+
assert(stream.currentSchema("value").dataType === ShortType)
52+
checkAnswer(stream.read(), Row(17))
53+
54+
withSQLConf(DeltaSQLConf.DELTA_SCHEMA_AUTO_MIGRATE.key -> "false") {
55+
stream.write(53)("CAST(value AS INT)")
56+
assert(stream.currentSchema("value").dataType === ShortType)
57+
checkAnswer(stream.read(), Row(17) :: Row(53) :: Nil)
58+
}
59+
}
60+
}
61+
62+
test("type isn't widened if type widening is disabled") {
63+
withDeltaStream[Int] { stream =>
64+
withSQLConf(DeltaConfigs.ENABLE_TYPE_WIDENING.defaultTablePropertyKey -> "false") {
65+
stream.write(17)("CAST(value AS SHORT)")
66+
assert(stream.currentSchema("value").dataType === ShortType)
67+
checkAnswer(stream.read(), Row(17))
68+
69+
stream.write(53)("CAST(value AS INT)")
70+
assert(stream.currentSchema("value").dataType === ShortType)
71+
checkAnswer(stream.read(), Row(17) :: Row(53) :: Nil)
72+
}
73+
}
74+
}
75+
76+
test("type is widened if type widening and schema evolution are enabled") {
77+
withDeltaStream[Int] { stream =>
78+
stream.write(17)("CAST(value AS SHORT)")
79+
assert(stream.currentSchema("value").dataType === ShortType)
80+
checkAnswer(stream.read(), Row(17))
81+
82+
stream.write(Int.MaxValue)("CAST(value AS INT)")
83+
assert(stream.currentSchema("value").dataType === IntegerType)
84+
checkAnswer(stream.read(), Row(17) :: Row(Int.MaxValue) :: Nil)
85+
}
86+
}
87+
88+
test("type can be widened even if type casting is disabled in the sink") {
89+
withDeltaStream[Int] { stream =>
90+
stream.write(17)("CAST(value AS SHORT)")
91+
assert(stream.currentSchema("value").dataType === ShortType)
92+
checkAnswer(stream.read(), Row(17))
93+
94+
withSQLConf(DeltaSQLConf.DELTA_STREAMING_SINK_ALLOW_IMPLICIT_CASTS.key -> "false") {
95+
stream.write(Int.MaxValue)("CAST(value AS INT)")
96+
assert(stream.currentSchema("value").dataType === IntegerType)
97+
checkAnswer(stream.read(), Row(17) :: Row(Int.MaxValue) :: Nil)
98+
}
99+
}
100+
}
101+
102+
test("type isn't changed if it's not a wider type") {
103+
withDeltaStream[Int] { stream =>
104+
stream.write(Int.MaxValue)("CAST(value AS INT)")
105+
assert(stream.currentSchema("value").dataType === IntegerType)
106+
checkAnswer(stream.read(), Row(Int.MaxValue))
107+
108+
stream.write(17)("CAST(value AS SHORT)")
109+
assert(stream.currentSchema("value").dataType === IntegerType)
110+
checkAnswer(stream.read(), Row(Int.MaxValue) :: Row(17) :: Nil)
111+
}
112+
}
113+
114+
test("type isn't changed if it's not eligible for automatic widening: int -> decimal") {
115+
withDeltaStream[Int] { stream =>
116+
stream.write(17)("CAST(value AS INT)")
117+
assert(stream.currentSchema("value").dataType === IntegerType)
118+
checkAnswer(stream.read(), Row(17))
119+
120+
stream.write(567)("CAST(value AS DECIMAL(20, 0))")
121+
assert(stream.currentSchema("value").dataType === IntegerType)
122+
checkAnswer(stream.read(), Row(17) :: Row(567) :: Nil)
123+
}
124+
}
125+
126+
test("type isn't changed if it's not eligible for automatic widening: int -> double") {
127+
withDeltaStream[Int] { stream =>
128+
stream.write(17)("CAST(value AS INT)")
129+
assert(stream.currentSchema("value").dataType === IntegerType)
130+
checkAnswer(stream.read(), Row(17))
131+
132+
stream.write(567)("CAST(value AS DOUBLE)")
133+
assert(stream.currentSchema("value").dataType === IntegerType)
134+
checkAnswer(stream.read(), Row(17) :: Row(567) :: Nil)
135+
}
136+
}
137+
138+
test("widen type and add a new column with schema evolution") {
139+
withDeltaStream[(Int, Int)] { stream =>
140+
stream.write((17, -1))("CAST(_1 AS SHORT) AS a")
141+
assert(stream.currentSchema === new StructType().add("a", ShortType))
142+
checkAnswer(stream.read(), Row(17))
143+
144+
stream.write((12, 3456))("CAST(_1 AS INT) AS a", "CAST(_2 AS DECIMAL(10, 2)) AS b")
145+
assert(stream.currentSchema === new StructType()
146+
.add("a", IntegerType, nullable = true,
147+
metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))
148+
.add("b", DecimalType(10, 2)))
149+
checkAnswer(stream.read(), Row(17, null) :: Row(12, 3456) :: Nil)
150+
}
151+
}
152+
153+
test("widen type during write with missing column") {
154+
withDeltaStream[(Int, Int)] { stream =>
155+
stream.write((17, 45))("CAST(_1 AS SHORT) AS a", "CAST(_2 AS LONG) AS b")
156+
assert(stream.currentSchema === new StructType()
157+
.add("a", ShortType)
158+
.add("b", LongType))
159+
checkAnswer(stream.read(), Row(17, 45))
160+
161+
stream.write((12, -1))("CAST(_1 AS INT) AS a")
162+
assert(stream.currentSchema === new StructType()
163+
.add("a", IntegerType, nullable = true,
164+
metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType))
165+
.add("b", LongType))
166+
checkAnswer(stream.read(), Row(17, 45) :: Row(12, null) :: Nil)
167+
}
168+
}
169+
170+
test("widen type after column rename and drop") {
171+
withDeltaStream[(Int, Int)] { stream =>
172+
stream.write((17, 45))("CAST(_1 AS SHORT) AS a", "CAST(_2 AS DECIMAL(10, 0)) AS b")
173+
assert(stream.currentSchema === new StructType()
174+
.add("a", ShortType)
175+
.add("b", DecimalType(10, 0)))
176+
checkAnswer(stream.read(), Row(17, 45))
177+
178+
sql(
179+
s"""
180+
|ALTER TABLE delta.`${stream.deltaLog.dataPath}` SET TBLPROPERTIES (
181+
| 'delta.columnMapping.mode' = 'name',
182+
| 'delta.minReaderVersion' = '2',
183+
| 'delta.minWriterVersion' = '5'
184+
|)
185+
""".stripMargin)
186+
sql(s"ALTER TABLE delta.`${stream.deltaLog.dataPath}` DROP COLUMN b")
187+
sql(s"ALTER TABLE delta.`${stream.deltaLog.dataPath}` RENAME COLUMN a to c")
188+
assert(stream.currentSchema === new StructType().add("c", ShortType))
189+
190+
stream.write((12, -1))("CAST(_1 AS INT) AS c")
191+
assert(stream.currentSchema === new StructType().add("c", IntegerType, nullable = true,
192+
metadata = typeWideningMetadata(version = 4, from = ShortType, to = IntegerType)))
193+
checkAnswer(stream.read(), Row(17) :: Row(12) :: Nil)
194+
}
195+
}
196+
197+
test("type widening in addBatch") {
198+
withTempDir { tempDir =>
199+
val tablePath = tempDir.getAbsolutePath
200+
val deltaLog = DeltaLog.forTable(spark, tablePath)
201+
sqlContext.sparkContext.setLocalProperty(StreamExecution.QUERY_ID_KEY, "streaming_query")
202+
val sink = DeltaSink(
203+
sqlContext,
204+
path = deltaLog.dataPath,
205+
partitionColumns = Seq.empty,
206+
outputMode = OutputMode.Append(),
207+
options = new DeltaOptions(options = Map.empty, conf = spark.sessionState.conf)
208+
)
209+
210+
val schema = new StructType().add("value", ShortType)
211+
212+
{
213+
val data = Seq(0, 1).toDF("value").selectExpr("CAST(value AS SHORT)")
214+
sink.addBatch(0, data)
215+
val df = spark.read.format("delta").load(tablePath)
216+
assert(df.schema === schema)
217+
checkAnswer(df, Row(0) :: Row(1) :: Nil)
218+
}
219+
{
220+
val data = Seq(2, 3).toDF("value").selectExpr("CAST(value AS INT)")
221+
sink.addBatch(1, data)
222+
val df = spark.read.format("delta").load(tablePath)
223+
assert(df.schema === new StructType().add("value", IntegerType, nullable = true,
224+
metadata = typeWideningMetadata(version = 1, from = ShortType, to = IntegerType)))
225+
checkAnswer(df, Row(0) :: Row(1) :: Row(2) :: Row(3) :: Nil)
226+
}
227+
}
228+
}
229+
}

0 commit comments

Comments
 (0)