Update Delta Lake table schema

Delta Lake lets you update the schema of a table. The following types of changes are supported:

  • Adding new columns (at arbitrary positions)

  • Reordering existing columns

  • Renaming existing columns

You can make these changes explicitly using DDL or implicitly using DML.

Important

An update to a Delta table schema is an operation that conflicts with all concurrent Delta write operations.

When you update a Delta table schema, streams that read from that table terminate. If you want the stream to continue you must restart it. For recommended methods, see Production considerations for Structured Streaming.

Explicitly update schema to add columns

ALTER TABLE table_name ADD COLUMNS (col_name data_type [COMMENT col_comment] [FIRST|AFTER colA_name], ...)

By default, nullability is true.

To add a column to a nested field, use:

ALTER TABLE table_name ADD COLUMNS (col_name.nested_col_name data_type [COMMENT col_comment] [FIRST|AFTER colA_name], ...)

For example, if the schema before running ALTER TABLE boxes ADD COLUMNS (colB.nested STRING AFTER field1) is:

- root
| - colA
| - colB
| +-field1
| +-field2

the schema after is:

- root
| - colA
| - colB
| +-field1
| +-nested
| +-field2

Note

Adding nested columns is supported only for structs. Arrays and maps are not supported.

Explicitly update schema to change column comment or ordering

ALTER TABLE table_name ALTER [COLUMN] col_name (COMMENT col_comment | FIRST | AFTER colA_name)

To change a column in a nested field, use:

ALTER TABLE table_name ALTER [COLUMN] col_name.nested_col_name (COMMENT col_comment | FIRST | AFTER colA_name)

For example, if the schema before running ALTER TABLE boxes ALTER COLUMN colB.field2 FIRST is:

- root
| - colA
| - colB
| +-field1
| +-field2

the schema after is:

- root
| - colA
| - colB
| +-field2
| +-field1

Explicitly update schema to replace columns

ALTER TABLE table_name REPLACE COLUMNS (col_name1 col_type1 [COMMENT col_comment1], ...)

For example, when running the following DDL:

ALTER TABLE boxes REPLACE COLUMNS (colC STRING, colB STRUCT<field2:STRING, nested:STRING, field1:STRING>, colA STRING)

if the schema before is:

- root
| - colA
| - colB
| +-field1
| +-field2

the schema after is:

- root
| - colC
| - colB
| +-field2
| +-nested
| +-field1
| - colA

Explicitly update schema to rename columns

Preview

This feature is in Public Preview.

Note

This feature is available in Databricks Runtime 10.2 and above.

To rename columns without rewriting any of the columns’ existing data, you must enable column mapping for the table. See Rename and drop columns with Delta Lake column mapping.

To rename a column:

ALTER TABLE table_name RENAME COLUMN old_col_name TO new_col_name

To rename a nested field:

ALTER TABLE table_name RENAME COLUMN col_name.old_nested_field TO new_nested_field

For example, when you run the following command:

ALTER TABLE boxes RENAME COLUMN colB.field1 TO field001

If the schema before is:

- root
| - colA
| - colB
| +-field1
| +-field2

Then the schema after is:

- root
| - colA
| - colB
| +-field001
| +-field2

See Rename and drop columns with Delta Lake column mapping.

Explicitly update schema to drop columns

Preview

This feature is in Public Preview.

Note

This feature is available in Databricks Runtime 11.0 and above.

To drop columns as a metadata-only operation without rewriting any data files, you must enable column mapping for the table. See Rename and drop columns with Delta Lake column mapping.

Important

Dropping a column from metadata does not delete the underlying data for the column in files. To purge the dropped column data, you can use REORG TABLE to rewrite files. You can then use VACUUM to physically delete the files that contain the dropped column data.

To drop a column:

ALTER TABLE table_name DROP COLUMN col_name

To drop multiple columns:

ALTER TABLE table_name DROP COLUMNS (col_name_1, col_name_2)

Explicitly update schema to change column type or name

You can change a column’s type or name or drop a column by rewriting the table. To do this, use the overwriteSchema option.

The following example shows changing a column type:

(spark.read.table(...)
  .withColumn("birthDate", col("birthDate").cast("date"))
  .write
  .mode("overwrite")
  .option("overwriteSchema", "true")
  .saveAsTable(...)
)

The following example shows changing a column name:

(spark.read.table(...)
  .withColumnRenamed("dateOfBirth", "birthDate")
  .write
  .mode("overwrite")
  .option("overwriteSchema", "true")
  .saveAsTable(...)
)

Add columns with automatic schema update

Columns that are present in the DataFrame but missing from the table are automatically added as part of a write transaction when:

  • write or writeStream have .option("mergeSchema", "true")

  • spark.databricks.delta.schema.autoMerge.enabled is true

When both options are specified, the option from the DataFrameWriter takes precedence. The added columns are appended to the end of the struct they are present in. Case is preserved when appending a new column.

Note

  • mergeSchema cannot be used with INSERT INTO or .write.insertInto().

Automatic schema evolution for Delta Lake merge

Schema evolution allows users to resolve schema mismatches between the target and source table in merge. It handles the following two cases:

  1. A column in the source table is not present in the target table. The new column is added to the target schema, and its values are inserted or updated using the source values.

  2. A column in the target table is not present in the source table. The target schema is left unchanged; the values in the additional target column are either left unchanged (for UPDATE) or set to NULL (for INSERT).

Important

To use schema evolution, you must set the Spark session configuration spark.databricks.delta.schema.autoMerge.enabled to true before you run the merge command.

Note

  • In Databricks Runtime 12.2 and above, columns present in the source table can be specified by name in insert or update actions. In Databricks Runtime 12.1 and below, only INSERT * or UPDATE SET * actions can be used for schema evolution with merge.

Here are a few examples of the effects of merge operation with and without schema evolution.

Columns

Query (in SQL)

Behavior without schema evolution (default)

Behavior with schema evolution

Target columns: key, value

Source columns: key, value, new_value

MERGE INTO target_table t
USING source_table s
ON t.key = s.key
WHEN MATCHED
  THEN UPDATE SET *
WHEN NOT MATCHED
  THEN INSERT *

The table schema remains unchanged; only columns key, value are updated/inserted.

The table schema is changed to (key, value, new_value). Existing records with matches are updated with the value and new_value in the source. New rows are inserted with the schema (key, value, new_value).

Target columns: key, old_value

Source columns: key, new_value

MERGE INTO target_table t
USING source_table s
ON t.key = s.key
WHEN MATCHED
  THEN UPDATE SET *
WHEN NOT MATCHED
  THEN INSERT *

UPDATE and INSERT actions throw an error because the target column old_value is not in the source.

The table schema is changed to (key, old_value, new_value). Existing records with matches are updated with the new_value in the source leaving old_value unchanged. New records are inserted with the specified key, new_value, and NULL for the old_value.

Target columns: key, old_value

Source columns: key, new_value

MERGE INTO target_table t
USING source_table s
ON t.key = s.key
WHEN MATCHED
  THEN UPDATE SET new_value = s.new_value

UPDATE throws an error because column new_value does not exist in the target table.

The table schema is changed to (key, old_value, new_value). Existing records with matches are updated with the new_value in the source leaving old_value unchanged, and unmatched records have NULL entered for new_value. See note (1).

Target columns: key, old_value

Source columns: key, new_value

MERGE INTO target_table t
USING source_table s
ON t.key = s.key
WHEN NOT MATCHED
  THEN INSERT (key, new_value) VALUES (s.key, s.new_value)

INSERT throws an error because column new_value does not exist in the target table.

The table schema is changed to (key, old_value, new_value). New records are inserted with the specified key, new_value, and NULL for the old_value. Existing records have NULL entered for new_value leaving old_value unchanged. See note (1).

(1) This behavior is available in Databricks Runtime 12.2 and above; Databricks Runtime 12.1 and below error in this condition.

Exclude columns with Delta Lake merge

In Databricks Runtime 12.0 and above, you can use EXCEPT clauses in merge conditions to explicitly exclude columns. The behavior of the EXCEPT keyword varies depending on whether or not schema evolution is enabled.

With schema evolution disabled, the EXCEPT keyword applies to the list of columns in the target table and allows excluding columns from UPDATE or INSERT actions. Excluded columns are set to null.

With schema evolution enabled, the EXCEPT keyword applies to the list of columns in the source table and allows excluding columns from schema evolution. A new column in the source that is not present in the target is not added to the target schema if it is listed in the EXCEPT clause. Excluded columns that are already present in the target are set to null.

The following examples demonstrate this syntax:

Columns

Query (in SQL)

Behavior without schema evolution (default)

Behavior with schema evolution

Target columns: id, title, last_updated

Source columns: id, title, review, last_updated

MERGE INTO target t
USING source s
ON t.id = s.id
WHEN MATCHED
  THEN UPDATE SET last_updated = current_date()
WHEN NOT MATCHED
  THEN INSERT * EXCEPT (last_updated)

Matched rows are updated by setting the last_updated field to the current date. New rows are inserted using values for id and title. The excluded field last_updated is set to null. The field review is ignored because it is not in the target.

Matched rows are updated by setting the last_updated field to the current date. Schema is evolved to add the field review. New rows are inserted using all source fields except last_updated which is set to null.

Target columns: id, title, last_updated

Source columns: id, title, review, internal_count

MERGE INTO target t
USING source s
ON t.id = s.id
WHEN MATCHED
  THEN UPDATE SET last_updated = current_date()
WHEN NOT MATCHED
  THEN INSERT * EXCEPT (last_updated, internal_count)

INSERT throws an error because column internal_count does not exist in the target table.

Matched rows are updated by setting the last_updated field to the current date. The review field is added to the target table, but the internal_count field is ignored. New rows inserted have last_updated set to null.

Automatic schema evolution for arrays of structs

Delta MERGE INTO supports resolving struct fields by name and evolving schemas for arrays of structs. With schema evolution enabled, target table schemas will evolve for arrays of structs, which also works with any nested structs inside of arrays.

Note

In Databricks Runtime 12.2 and above, struct fields present in the source table can be specified by name in insert or update commands. In Databricks Runtime 12.1 and below, only INSERT * or UPDATE SET * commands can be used for schema evolution with merge.

Here are a few examples of the effects of merge operations with and without schema evolution for arrays of structs.

Source schema

Target schema

Behavior without schema evolution (default)

Behavior with schema evolution

array<struct<b: string, a: string>>

array<struct<a: int, b: int>>

The table schema remains unchanged. Columns will be resolved by name and updated or inserted.

The table schema remains unchanged. Columns will be resolved by name and updated or inserted.

array<struct<a: int, c: string, d: string>>

array<struct<a: string, b: string>>

update and insert throw errors because c and d do not exist in the target table.

The table schema is changed to array<struct<a: string, b: string, c: string, d: string>>. c and d are inserted as NULL for existing entries in the target table. update and insert fill entries in the source table with a casted to string and b as NULL.

array<struct<a: string, b: struct<c: string, d: string>>>

array<struct<a: string, b: struct<c: string>>>

update and insert throw errors because d does not exist in the target table.

The target table schema is changed to array<struct<a: string, b: struct<c: string, d: string>>>. d is inserted as NULL for existing entries in the target table.

Dealing with NullType columns in schema updates

Because Parquet doesn’t support NullType, NullType columns are dropped from the DataFrame when writing into Delta tables, but are still stored in the schema. When a different data type is received for that column, Delta Lake merges the schema to the new data type. If Delta Lake receives a NullType for an existing column, the old schema is retained and the new column is dropped during the write.

NullType in streaming is not supported. Since you must set schemas when using streaming this should be very rare. NullType is also not accepted for complex types such as ArrayType and MapType.

Replace table schema

By default, overwriting the data in a table does not overwrite the schema. When overwriting a table using mode("overwrite") without replaceWhere, you may still want to overwrite the schema of the data being written. You replace the schema and partitioning of the table by setting the overwriteSchema option to true:

df.write.option("overwriteSchema", "true")

Important

You cannot specify overwriteSchema as true when using dynamic partition overwrite.