|
| 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