Skip to content

Commit 9b3fa0a

Browse files
johanl-dbvkorukanti
authored andcommitted
Type Widening in ALTER TABLE CHANGE COLUMN
## Description This change introduces the `typeWidening` delta table feature, allowing to widen the type of existing columns and fields in a delta table using the `ALTER TABLE CHANGE COLUMN TYPE` or `ALTER TABLE REPLACE COLUMNS` commands. The table feature is introduced as `typeWidening-dev` during implementation and is available in testing only. For now, only byte -> short -> int are supported. Other changes will require support in the Spark parquet reader that will be introduced in Spark 4.0 Type widening feature request: #2622 Type Widening protocol RFC: #2624 A new test suite `DeltaTypeWideningSuite` is created, containing: - `DeltaTypeWideningAlterTableTests`: Covers applying supported and unsupported type changes on partitioned columns, non-partitioned columns and nested fields - `DeltaTypeWideningTableFeatureTests`: Covers adding the `typeWidening` table feature ## This PR introduces the following *user-facing* changes The table feature is available in testing only, there's no user-facing changes as of now. The type widening table feature will introduce the following changes: - Adding the `typeWidening` via a table property: ``` ALTER TABLE t SET TBLPROPERTIES (‘delta.enableTypeWidening' = true) ``` - Apply a widening type change: ``` ALTER TABLE t CHANGE COLUMN int_col TYPE long ``` or ``` ALTER TABLE t REPLACE COLUMNS int_col TYPE long ``` Note: both ALTER TABLE commands reuse the existing syntax for setting a table property and applying a type change, no new SQL syntax is being introduced by this feature. Closes #2645 GitOrigin-RevId: 2ca0e6b22ec24b304241460553547d0d4c6026a2
1 parent eb59d4a commit 9b3fa0a

File tree

6 files changed

+586
-4
lines changed

6 files changed

+586
-4
lines changed

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

+11
Original file line numberDiff line numberDiff line change
@@ -725,6 +725,17 @@ trait DeltaConfigsBase extends DeltaLogging {
725725
"needs to be a boolean."
726726
)
727727

728+
/**
729+
* Whether widening the type of an existing column or field is allowed, either manually using
730+
* ALTER TABLE CHANGE COLUMN or automatically if automatic schema evolution is enabled.
731+
*/
732+
val ENABLE_TYPE_WIDENING = buildConfig[Boolean](
733+
key = "enableTypeWidening",
734+
defaultValue = false.toString,
735+
fromString = _.toBoolean,
736+
validationFunction = _ => true,
737+
helpMessage = "needs to be a boolean.")
738+
728739
val MANAGED_COMMIT_OWNER_NAME = buildConfig[Option[String]](
729740
"managedCommits.commitOwner-dev",
730741
null,

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

+14-1
Original file line numberDiff line numberDiff line change
@@ -355,7 +355,8 @@ object TableFeature {
355355
// managed-commits are under development and only available in testing.
356356
ManagedCommitTableFeature,
357357
// Row IDs are still under development and only available in testing.
358-
RowTrackingFeature)
358+
RowTrackingFeature,
359+
TypeWideningTableFeature)
359360
}
360361
val featureMap = features.map(f => f.name.toLowerCase(Locale.ROOT) -> f).toMap
361362
require(features.size == featureMap.size, "Lowercase feature names must not duplicate.")
@@ -625,6 +626,18 @@ object ManagedCommitTableFeature
625626
}
626627
}
627628

629+
object TypeWideningTableFeature extends ReaderWriterFeature(name = "typeWidening-dev")
630+
with FeatureAutomaticallyEnabledByMetadata {
631+
override def automaticallyUpdateProtocolOfExistingTables: Boolean = true
632+
633+
private def isTypeWideningSupportNeededByMetadata(metadata: Metadata): Boolean =
634+
DeltaConfigs.ENABLE_TYPE_WIDENING.fromMetaData(metadata)
635+
636+
override def metadataRequiresFeatureToBeEnabled(
637+
metadata: Metadata,
638+
spark: SparkSession): Boolean = isTypeWideningSupportNeededByMetadata(metadata)
639+
}
640+
628641
/**
629642
* Features below are for testing only, and are being registered to the system only in the testing
630643
* environment. See [[TableFeature.allSupportedFeaturesMap]] for the registration.
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,63 @@
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
18+
19+
import org.apache.spark.sql.delta.actions.{Metadata, Protocol, TableFeatureProtocolUtils}
20+
21+
import org.apache.spark.sql.catalyst.expressions.Cast
22+
import org.apache.spark.sql.types._
23+
24+
object TypeWidening {
25+
26+
/**
27+
* Returns whether the protocol version supports the Type Widening table feature.
28+
*/
29+
def isSupported(protocol: Protocol): Boolean =
30+
protocol.isFeatureSupported(TypeWideningTableFeature)
31+
32+
/**
33+
* Returns whether Type Widening is enabled on this table version. Checks that Type Widening is
34+
* supported, which is a pre-requisite for enabling Type Widening, throws an error if
35+
* not. When Type Widening is enabled, the type of existing columns or fields can be widened
36+
* using ALTER TABLE CHANGE COLUMN.
37+
*/
38+
def isEnabled(protocol: Protocol, metadata: Metadata): Boolean = {
39+
val isEnabled = DeltaConfigs.ENABLE_TYPE_WIDENING.fromMetaData(metadata)
40+
if (isEnabled && !isSupported(protocol)) {
41+
throw new IllegalStateException(
42+
s"Table property '${DeltaConfigs.ENABLE_TYPE_WIDENING.key}' is " +
43+
s"set on the table but this table version doesn't support table feature " +
44+
s"'${TableFeatureProtocolUtils.propertyKey(TypeWideningTableFeature)}'.")
45+
}
46+
isEnabled
47+
}
48+
49+
/**
50+
* Returns whether the given type change is eligible for widening. This only checks atomic types.
51+
* It is the responsibility of the caller to recurse into structs, maps and arrays.
52+
*/
53+
def isTypeChangeSupported(fromType: AtomicType, toType: AtomicType): Boolean =
54+
(fromType, toType) match {
55+
case (from, to) if from == to => true
56+
// All supported type changes below are supposed to be widening, but to be safe, reject any
57+
// non-widening change upfront.
58+
case (from, to) if !Cast.canUpCast(from, to) => false
59+
case (ByteType, ShortType) => true
60+
case (ByteType | ShortType, IntegerType) => true
61+
case _ => false
62+
}
63+
}

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

+3-1
Original file line numberDiff line numberDiff line change
@@ -723,7 +723,8 @@ case class AlterTableChangeColumnDeltaCommand(
723723
newType,
724724
resolver,
725725
txn.metadata.columnMappingMode,
726-
columnPath :+ originalField.name
726+
columnPath :+ originalField.name,
727+
allowTypeWidening = TypeWidening.isEnabled(txn.protocol, txn.metadata)
727728
).nonEmpty) {
728729
throw DeltaErrors.alterTableChangeColumnException(
729730
fieldPath = UnresolvedAttribute(columnPath :+ originalField.name).name,
@@ -802,6 +803,7 @@ case class AlterTableReplaceColumnsDeltaCommand(
802803
changingSchema,
803804
resolver,
804805
txn.metadata.columnMappingMode,
806+
allowTypeWidening = TypeWidening.isEnabled(txn.protocol, txn.metadata),
805807
failOnAmbiguousChanges = true
806808
).foreach { operation =>
807809
throw DeltaErrors.alterTableReplaceColumnsException(

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

+10-2
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import scala.collection.mutable
2121
import scala.collection.mutable.ArrayBuffer
2222
import scala.util.control.NonFatal
2323

24-
import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaColumnMappingMode, DeltaErrors, DeltaLog, GeneratedColumn, NoMapping}
24+
import org.apache.spark.sql.delta.{DeltaAnalysisException, DeltaColumnMappingMode, DeltaErrors, DeltaLog, GeneratedColumn, NoMapping, TypeWidening}
2525
import org.apache.spark.sql.delta.actions.Protocol
2626
import org.apache.spark.sql.delta.commands.cdc.CDCReader
2727
import org.apache.spark.sql.delta.metering.DeltaLogging
@@ -911,6 +911,8 @@ def normalizeColumnNamesInDataType(
911911
* @param failOnAmbiguousChanges Throw an error if a StructField both has columns dropped and new
912912
* columns added. These are ambiguous changes, because we don't
913913
* know if a column needs to be renamed, dropped, or added.
914+
* @param allowTypeWidening Whether widening type changes as defined in [[TypeWidening]]
915+
* can be applied.
914916
* @return None if the data types can be changed, otherwise Some(err) containing the reason.
915917
*/
916918
def canChangeDataType(
@@ -919,7 +921,8 @@ def normalizeColumnNamesInDataType(
919921
resolver: Resolver,
920922
columnMappingMode: DeltaColumnMappingMode,
921923
columnPath: Seq[String] = Nil,
922-
failOnAmbiguousChanges: Boolean = false): Option[String] = {
924+
failOnAmbiguousChanges: Boolean = false,
925+
allowTypeWidening: Boolean = false): Option[String] = {
923926
def verify(cond: Boolean, err: => String): Unit = {
924927
if (!cond) {
925928
throw DeltaErrors.cannotChangeDataType(err)
@@ -970,6 +973,11 @@ def normalizeColumnNamesInDataType(
970973
(if (columnPath.nonEmpty) s" from $columnName" else ""))
971974
}
972975

976+
case (fromDataType: AtomicType, toDataType: AtomicType) if allowTypeWidening =>
977+
verify(TypeWidening.isTypeChangeSupported(fromDataType, toDataType),
978+
s"changing data type of ${UnresolvedAttribute(columnPath).name} " +
979+
s"from $fromDataType to $toDataType")
980+
973981
case (fromDataType, toDataType) =>
974982
verify(fromDataType == toDataType,
975983
s"changing data type of ${UnresolvedAttribute(columnPath).name} " +

0 commit comments

Comments
 (0)