Note
Access to this page requires authorization. You can try signing in or changing directories.
Access to this page requires authorization. You can try changing directories.
Important
This feature is in Public Preview in Databricks Runtime 15.4 LTS and above.
Tables with type widening enabled allow you to change column data types to a wider type without rewriting underlying data files. You can either change column types manually or use schema evolution to evolve column types.
Important
Type widening is available in Databricks Runtime 15.4 LTS and above. Tables with type widening enabled can only be read in Databricks Runtime 15.4 LTS and above.
Type widening requires Delta Lake. All Unity Catalog managed tables use Delta Lake by default.
Supported type changes
You can widen types according to the following rules:
Source type | Supported wider types |
---|---|
byte |
short , int , long , decimal , double |
short |
int , long , decimal , double |
int |
long , decimal , double |
long |
decimal |
float |
double |
decimal |
decimal with greater precision and scale |
date |
timestampNTZ |
To avoid accidentally promoting integer values to decimals, you must manually commit type changes from byte
, short
, int
, or long
to decimal
or double
. When promoting an integer type to decimal
or double
, if any downstream ingestion writes this value back to an integer column, Spark will truncate the fractional part of the values by default.
Note
When changing any numeric type to decimal
, the total precision must be equal to or greater than the starting precision. If you also increase the scale, the total precision must increase by a corresponding amount.
The minimum target for byte
, short
, and int
types is decimal(10,0)
. The minimum target for long
is decimal(20,0)
.
If you want to add two decimal places to a field with decimal(10,1)
, the minimum target is decimal(12,3)
.
Type changes are supported for top-level columns and fields nested inside structs, maps, and arrays.
Enable type widening
You can enable type widening on an existing table by setting the delta.enableTypeWidening
table property to true
:
ALTER TABLE <table_name> SET TBLPROPERTIES ('delta.enableTypeWidening' = 'true')
You can also enable type widening during table creation:
CREATE TABLE T(c1 INT) TBLPROPERTIES('delta.enableTypeWidening' = 'true')
Important
When you enable type widening, it sets the table feature typeWidening
, which upgrades the reader and writer protocols. You must use Databricks Runtime 15.4 or above for to interact with tables with type widening enabled. If external clients also interact with the table, verify that they support this table feature. See Delta Lake feature compatibility and protocols.
Manually apply a type change
Use the ALTER COLUMN
command to manually change types:
ALTER TABLE <table_name> ALTER COLUMN <col_name> TYPE <new_type>
This operation updates the table schema without rewriting the underlying data files.
Widen types with automatic schema evolution
Schema evolution works with type widening to update data types in target tables to match the type of incoming data.
Note
Without type widening enabled, schema evolution always attempts to downcast data to match column types in the target table. If you don’t want to automatically widen data types in your target tables, disable type widening before you run workloads with schema evolution enabled.
To use schema evolution to widen the data type of a column during ingestion, you must meet the following conditions:
- The write command runs with automatic schema evolution enabled.
- The target table has type widening enabled.
- The source column type is wider than the target column type.
- Type widening supports the type change.
- The type change is not one of
byte
,short
,int
, orlong
todecimal
ordouble
. These type changes can only be applied manually using ALTER TABLE to avoid accidental promotion of integers to decimals.
Type mismatches that don't meet all of these conditions follow normal schema enforcement rules. See Schema enforcement.
Disable the type widening table feature
You can prevent accidental type widening on enabled tables by setting the property to false
:
ALTER TABLE <table_name> SET TBLPROPERTIES ('delta.enableTypeWidening' = 'false')
This setting prevents future type changes to the table, but doesn't remove the type widening table feature or undo types that have changed.
If you need to completely remove the type widening table features, you can use the DROP FEATURE
command as shown in the following example:
ALTER TABLE <table-name> DROP FEATURE 'typeWidening' [TRUNCATE HISTORY]
Note
Tables that enabled type widening using Databricks Runtime 15.4 LTS require dropping feature typeWidening-preview
instead.
When dropping type widening, all data files that don’t conform to the current table schema are rewritten. See Drop a Delta Lake table feature and downgrade table protocol.
Streaming from a Delta table
Note
Support for Type Widening in Structured Streaming is available in Databricks Runtime 16.3 and above.
When streaming from a Delta table, type changes are treated as non-additive schema changes, similar to renaming or dropping a column with Column Mapping.
You can provide a schema tracking location to enable streaming from Delta Lake tables with a type change applied.
Each streaming read against a data source must have its own schemaTrackingLocation
specified. The specified schemaTrackingLocation
must be contained in the directory specified for the checkpointLocation
of the target table for streaming write.
Note
For streaming workloads that combine data from multiple source Delta tables, you must specify unique directories in the checkpointLocation
for each source table.
The option schemaTrackingLocation
is used to specify the path for schema tracking, as shown in the following code example:
Python
checkpoint_path = "/path/to/checkpointLocation"
(spark.readStream
.option("schemaTrackingLocation", checkpoint_path)
.table("delta_source_table")
.writeStream
.option("checkpointLocation", checkpoint_path)
.toTable("output_table")
)
Scala
val checkpointPath = "/path/to/checkpointLocation"
spark.readStream
.option("schemaTrackingLocation", checkpointPath)
.table("delta_source_table")
.writeStream
.option("checkpointLocation", checkpointPath)
.toTable("output_table")
After providing a schema tracking location, the stream will evolve its tracked schema whenever a type change is detected and then stop. At that time, you can take any necessary action to handle the type change, such as enabling type widening on the downstream table or updating the streaming query.
To resume processing, set the Spark configuration spark.databricks.delta.streaming.allowSourceColumnTypeChange
or the DataFrame reader option allowSourceColumnTypeChange
:
Python
checkpoint_path = "/path/to/checkpointLocation"
(spark.readStream
.option("schemaTrackingLocation", checkpoint_path)
.option("allowSourceColumnTypeChange", "<delta_source_table_version>")
# alternatively to allow all future type changes for this stream:
# .option("allowSourceColumnTypeChange", "always")
.table("delta_source_table")
.writeStream
.option("checkpointLocation", checkpoint_path)
.toTable("output_table")
)
Scala
val checkpointPath = "/path/to/checkpointLocation"
spark.readStream
.option("schemaTrackingLocation", checkpointPath)
.option("allowSourceColumnTypeChange", "<delta_source_table_version>")
// alternatively to allow all future type changes for this stream:
// .option("allowSourceColumnTypeChange", "always")
.table("delta_source_table")
.writeStream
.option("checkpointLocation", checkpointPath)
.toTable("output_table")
SQL
-- To unblock for this particular stream just for this series of schema change(s):
SET spark.databricks.delta.streaming.allowSourceColumnTypeChange.ckpt_<checkpoint_id> = "<delta_source_table_version>"
-- To unblock for this particular stream:
SET spark.databricks.delta.streaming.allowSourceColumnTypeChange = "<delta_source_table_version>"
-- To unblock for all streams:
SET spark.databricks.delta.streaming.allowSourceColumnTypeChange = "always"
The checkpoint ID <checkpoint_id>
and Delta Lake source table version <delta_source_table_version>
are displayed in the error message when the stream stops.
Delta Sharing
Note
Support for Type Widening in Delta Sharing is available in Databricks Runtime 16.1 and above.
Sharing a Delta Lake table with type widening enabled is supported in Databricks-to-Databricks Delta Sharing. The provider and recipient must be on Databricks Runtime 16.1 or above.
To read Change Data Feed from a Delta Lake table with type widening enabled using Delta Sharing, you must set the response format to delta
:
spark.read
.format("deltaSharing")
.option("responseFormat", "delta")
.option("readChangeFeed", "true")
.option("startingVersion", "<start version>")
.option("endingVersion", "<end version>")
.load("<table>")
Reading Change Data Feed across type changes is not supported. You must instead split the operation into two separate reads, one ending at the table version containing the type change, and the other starting at the version containing the type change.
Limitations
Iceberg Compatibility
Iceberg doesn’t support all type changes covered by type widening, see Iceberg Schema Evolution. In particular, Azure Databricks does not support the following type changes:
byte
,short
,int
,long
todecimal
ordouble
- decimal scale increase
date
totimestampNTZ
When UniForm with Iceberg compatibility is enabled on a Delta Lake table, applying one of these type changes results in an error.
If you apply one of these unsupported type changes to a Delta Lake table, enabling Uniform with Iceberg compatibility on the table results in an error. To resolve the error, you must drop the type widening table feature.
Other Limitations
- Providing a schema tracking location using SQL when streaming from a Delta Lake table with a type change is not supported.
- Sharing a table using Delta Sharing with type widening enabled with non-Databricks consumers is not supported.