Table batch reads and writes
Delta Lake supports most of the options provided by Apache Spark DataFrame read and write APIs for performing batch reads and writes on tables.
For many Delta Lake operations on tables, you enable integration with Apache Spark DataSourceV2 and Catalog APIs (since 3.0) by setting configurations when you create a new SparkSession
. See Configure SparkSession.
In this article:
Create a table
Delta Lake supports creating two types of tables—tables defined in the metastore and tables defined by path.
To work with metastore-defined tables, you must enable integration with Apache Spark DataSourceV2 and Catalog APIs by setting configurations when you create a new SparkSession
. See Configure SparkSession.
You can create tables in the following ways.
SQL DDL commands: You can use standard SQL DDL commands supported in Apache Spark (for example,
CREATE TABLE
andREPLACE TABLE
) to create Delta tables.CREATE TABLE IF NOT EXISTS default.people10m ( id INT, firstName STRING, middleName STRING, lastName STRING, gender STRING, birthDate TIMESTAMP, ssn STRING, salary INT ) USING DELTA CREATE OR REPLACE TABLE default.people10m ( id INT, firstName STRING, middleName STRING, lastName STRING, gender STRING, birthDate TIMESTAMP, ssn STRING, salary INT ) USING DELTA
SQL also supports creating a table at a path, without creating an entry in the Hive metastore.
-- Create or replace table with path CREATE OR REPLACE TABLE delta.`/tmp/delta/people10m` ( id INT, firstName STRING, middleName STRING, lastName STRING, gender STRING, birthDate TIMESTAMP, ssn STRING, salary INT ) USING DELTA
DataFrameWriter
API: If you want to simultaneously create a table and insert data into it from Spark DataFrames or Datasets, you can use the SparkDataFrameWriter
(Scala or Java and Python).# Create table in the metastore using DataFrame's schema and write data to it df.write.format("delta").saveAsTable("default.people10m") # Create or replace partitioned table with path using DataFrame's schema and write/overwrite data to it df.write.format("delta").mode("overwrite").save("/tmp/delta/people10m")
// Create table in the metastore using DataFrame's schema and write data to it df.write.format("delta").saveAsTable("default.people10m") // Create table with path using DataFrame's schema and write data to it df.write.format("delta").mode("overwrite").save("/tmp/delta/people10m")
You can also create Delta tables using the Spark
DataFrameWriterV2
API.DeltaTableBuilder
API: You can also use theDeltaTableBuilder
API in Delta Lake to create tables. Compared to the DataFrameWriter APIs, this API makes it easier to specify additional information like column comments, table properties, and generated columns.Note
This feature is new and is in Preview.
# Create table in the metastore DeltaTable.createIfNotExists(spark) \ .tableName("default.people10m") \ .addColumn("id", "INT") \ .addColumn("firstName", "STRING") \ .addColumn("middleName", "STRING") \ .addColumn("lastName", "STRING", comment = "surname") \ .addColumn("gender", "STRING") \ .addColumn("birthDate", "TIMESTAMP") \ .addColumn("ssn", "STRING") \ .addColumn("salary", "INT") \ .execute() # Create or replace table with path and add properties DeltaTable.createOrReplace(spark) \ .addColumn("id", "INT") \ .addColumn("firstName", "STRING") \ .addColumn("middleName", "STRING") \ .addColumn("lastName", "STRING", comment = "surname") \ .addColumn("gender", "STRING") \ .addColumn("birthDate", "TIMESTAMP") \ .addColumn("ssn", "STRING") \ .addColumn("salary", "INT") \ .property("description", "table with people data") \ .location("/tmp/delta/people10m") \ .execute()
// Create table in the metastore DeltaTable.createOrReplace(spark) .tableName("default.people10m") .addColumn("id", "INT") .addColumn("firstName", "STRING") .addColumn("middleName", "STRING") .addColumn( DeltaTable.columnBuilder("lastName") .dataType("STRING") .comment("surname") .build()) .addColumn("lastName", "STRING", comment = "surname") .addColumn("gender", "STRING") .addColumn("birthDate", "TIMESTAMP") .addColumn("ssn", "STRING") .addColumn("salary", "INT") .execute() // Create or replace table with path and add properties DeltaTable.createOrReplace(spark) .addColumn("id", "INT") .addColumn("firstName", "STRING") .addColumn("middleName", "STRING") .addColumn( DeltaTable.columnBuilder("lastName") .dataType("STRING") .comment("surname") .build()) .addColumn("lastName", "STRING", comment = "surname") .addColumn("gender", "STRING") .addColumn("birthDate", "TIMESTAMP") .addColumn("ssn", "STRING") .addColumn("salary", "INT") .property("description", "table with people data") .location("/tmp/delta/people10m") .execute()
See the API documentation for details.
Partition data
You can partition data to speed up queries or DML that have predicates involving the partition columns. To partition data when you create a Delta table, specify a partition by columns. The following example partitions by gender.
-- Create table in the metastore
CREATE TABLE default.people10m (
id INT,
firstName STRING,
middleName STRING,
lastName STRING,
gender STRING,
birthDate TIMESTAMP,
ssn STRING,
salary INT
)
USING DELTA
PARTITIONED BY (gender)
df.write.format("delta").partitionBy("gender").saveAsTable("default.people10m")
DeltaTable.create(spark) \
.tableName("default.people10m") \
.addColumn("id", "INT") \
.addColumn("firstName", "STRING") \
.addColumn("middleName", "STRING") \
.addColumn("lastName", "STRING", comment = "surname") \
.addColumn("gender", "STRING") \
.addColumn("birthDate", "TIMESTAMP") \
.addColumn("ssn", "STRING") \
.addColumn("salary", "INT") \
.partitionedBy("gender") \
.execute()
df.write.format("delta").partitionBy("gender").saveAsTable("default.people10m")
DeltaTable.createOrReplace(spark)
.tableName("default.people10m")
.addColumn("id", "INT")
.addColumn("firstName", "STRING")
.addColumn("middleName", "STRING")
.addColumn(
DeltaTable.columnBuilder("lastName")
.dataType("STRING")
.comment("surname")
.build())
.addColumn("lastName", "STRING", comment = "surname")
.addColumn("gender", "STRING")
.addColumn("birthDate", "TIMESTAMP")
.addColumn("ssn", "STRING")
.addColumn("salary", "INT")
.partitionedBy("gender")
.execute()
To determine whether a table contains a specific partition, use the statement SELECT COUNT(*) > 0 FROM <table-name> WHERE <partition-column> = <value>
. If the partition exists, true
is returned. For example:
SELECT COUNT(*) > 0 AS `Partition exists` FROM default.people10m WHERE gender = "M"
display(spark.sql("SELECT COUNT(*) > 0 AS `Partition exists` FROM default.people10m WHERE gender = 'M'"))
display(spark.sql("SELECT COUNT(*) > 0 AS `Partition exists` FROM default.people10m WHERE gender = 'M'"))
Control data location
For tables defined in the metastore, you can optionally specify the LOCATION
as a path. Tables created with a specified LOCATION
are considered unmanaged by the metastore. Unlike a managed table, where no path is specified, an unmanaged table’s files are not deleted when you DROP
the table.
When you run CREATE TABLE
with a LOCATION
that already contains data stored using Delta Lake, Delta Lake does the following:
If you specify only the table name and location, for example:
CREATE TABLE default.people10m USING DELTA LOCATION '/tmp/delta/people10m'
the table in the metastore automatically inherits the schema, partitioning, and table properties of the existing data. This functionality can be used to “import” data into the metastore.
If you specify any configuration (schema, partitioning, or table properties), Delta Lake verifies that the specification exactly matches the configuration of the existing data.
Important
If the specified configuration does not exactly match the configuration of the data, Delta Lake throws an exception that describes the discrepancy.
Note
The metastore is not the source of truth about the latest information of a Delta table. In fact, the table definition in the metastore may not contain all the metadata like schema and properties. It contains the location of the table, and the table’s transaction log at the location is the source of truth. If you query the metastore from a system that is not aware of this Delta-specific customization, you may see incomplete or stale table information.
Use generated columns
Note
This feature is new and is in Preview.
Delta Lake supports generated columns which are a special type of columns whose values are automatically generated based on a user-specified function over other columns in the Delta table. When you write to a table with generated columns and you do not explicitly provide values for them, Delta Lake automatically computes the values. For example, you can automatically generate a date column (for partitioning the table by date) from the timestamp column; any writes into the table need only specify the data for the timestamp column. However, if you explicitly provide values for them, the values must satisfy the constraint (<value> <=> <generation expression>) IS TRUE
or the write will fail with an error.
Important
Tables created with generated columns have a higher table writer protocol version than the default. See Table protocol versioning to understand table protocol versioning and what it means to have a higher version of a table protocol version.
The following example shows how to create a table with generated columns:
DeltaTable.create(spark) \
.tableName("default.people10m") \
.addColumn("id", "INT") \
.addColumn("firstName", "STRING") \
.addColumn("middleName", "STRING") \
.addColumn("lastName", "STRING", comment = "surname") \
.addColumn("gender", "STRING") \
.addColumn("birthDate", "TIMESTAMP") \
.addColumn("dateOfBirth", DateType(), generatedAlwaysAs="CAST(birthDate AS DATE)") \
.addColumn("ssn", "STRING") \
.addColumn("salary", "INT") \
.partitionedBy("gender") \
.execute()
DeltaTable.create(spark)
.tableName("default.people10m")
.addColumn("id", "INT")
.addColumn("firstName", "STRING")
.addColumn("middleName", "STRING")
.addColumn(
DeltaTable.columnBuilder("lastName")
.dataType("STRING")
.comment("surname")
.build())
.addColumn("lastName", "STRING", comment = "surname")
.addColumn("gender", "STRING")
.addColumn("birthDate", "TIMESTAMP")
.addColumn(
DeltaTable.columnBuilder("dateOfBirth")
.dataType(DateType)
.generatedAlwaysAs("CAST(dateOfBirth AS DATE)")
.build())
.addColumn("ssn", "STRING")
.addColumn("salary", "INT")
.partitionedBy("gender")
.execute()
Generated columns are stored as if they were normal columns. That is, they occupy storage.
The following restrictions apply to generated columns:
A generation expression can use any SQL functions in Spark that always return the same result when given the same argument values, except the following types of functions:
User-defined functions.
Aggregate functions.
Window functions.
Functions returning multiple rows.
For Delta Lake 1.1.0 and above,
MERGE
operations support generated columns when you setspark.databricks.delta.schema.autoMerge.enabled
to true.
Delta Lake may be able to generate partition filters for a query whenever a partition column is defined by one of the following expressions:
CAST(col AS DATE)
and the type ofcol
isTIMESTAMP
.YEAR(col)
and the type ofcol
isTIMESTAMP
.Two partition columns defined by
YEAR(col), MONTH(col)
and the type ofcol
isTIMESTAMP
.Three partition columns defined by
YEAR(col), MONTH(col), DAY(col)
and the type ofcol
isTIMESTAMP
.Four partition columns defined by
YEAR(col), MONTH(col), DAY(col), HOUR(col)
and the type ofcol
isTIMESTAMP
.SUBSTRING(col, pos, len)
and the type ofcol
isSTRING
DATE_FORMAT(col, format)
and the type ofcol
isTIMESTAMP
.
If a partition column is defined by one of the preceding expressions, and a query filters data using the underlying base column of a generation expression, Delta Lake looks at the relationship between the base column and the generated column, and populates partition filters based on the generated partition column if possible. For example, given the following table:
DeltaTable.create(spark) \
.tableName("default.events") \
.addColumn("eventId", "BIGINT") \
.addColumn("data", "STRING") \
.addColumn("eventType", "STRING") \
.addColumn("eventTime", "TIMESTAMP") \
.addColumn("eventDate", "DATE", generatedAlwaysAs="CAST(eventTime AS DATE)") \
.partitionedBy("eventType", "eventDate") \
.execute()
If you then run the following query:
spark.sql('SELECT * FROM default.events WHERE eventTime >= "2020-10-01 00:00:00" <= "2020-10-01 12:00:00"')
Delta Lake automatically generates a partition filter so that the preceding query only reads the data in partition date=2020-10-01
even if a partition filter is not specified.
As another example, given the following table:
DeltaTable.create(spark) \
.tableName("default.events") \
.addColumn("eventId", "BIGINT") \
.addColumn("data", "STRING") \
.addColumn("eventType", "STRING") \
.addColumn("eventTime", "TIMESTAMP") \
.addColumn("year", "INT", generatedAlwaysAs="YEAR(eventTime)") \
.addColumn("month", "INT", generatedAlwaysAs="MONTH(eventTime)") \
.addColumn("day", "INT", generatedAlwaysAs="DAY(eventTime)") \
.partitionedBy("eventType", "year", "month", "day") \
.execute()
If you then run the following query:
spark.sql('SELECT * FROM default.events WHERE eventTime >= "2020-10-01 00:00:00" <= "2020-10-01 12:00:00"')
Delta Lake automatically generates a partition filter so that the preceding query only reads the data in partition year=2020/month=10/day=01
even if a partition filter is not specified.
You can use an EXPLAIN clause and check the provided plan to see whether Delta Lake automatically generates any partition filters.
Use special characters in column names
By default, special characters such as spaces and any of the characters ,;{}()\n\t=
are not supported in table column names. To include these special characters in a table’s column name, enable column mapping.
Read a table
You can load a Delta table as a DataFrame by specifying a table name or a path:
SELECT * FROM default.people10m -- query table in the metastore
SELECT * FROM delta.`/tmp/delta/people10m` -- query table by path
spark.table("default.people10m") # query table in the metastore
spark.read.format("delta").load("/tmp/delta/people10m") # query table by path
spark.table("default.people10m") // query table in the metastore
spark.read.format("delta").load("/tmp/delta/people10m") // create table by path
import io.delta.implicits._
spark.read.delta("/tmp/delta/people10m")
The DataFrame returned automatically reads the most recent snapshot of the table for any query; you never need to run REFRESH TABLE
. Delta Lake automatically uses partitioning and statistics to read the minimum amount of data when there are applicable predicates in the query.
Query an older snapshot of a table (time travel)
In this section:
Delta Lake time travel allows you to query an older snapshot of a Delta table. Time travel has many use cases, including:
Re-creating analyses, reports, or outputs (for example, the output of a machine learning model). This could be useful for debugging or auditing, especially in regulated industries.
Writing complex temporal queries.
Fixing mistakes in your data.
Providing snapshot isolation for a set of queries for fast changing tables.
This section describes the supported methods for querying older versions of tables, data retention concerns, and provides examples.
Syntax
This section shows how to query an older version of a Delta table.
DataFrameReader options
DataFrameReader options allow you to create a DataFrame from a Delta table that is fixed to a specific version of the table.
df1 = spark.read.format("delta").option("timestampAsOf", timestamp_string).load("/tmp/delta/people10m")
df2 = spark.read.format("delta").option("versionAsOf", version).load("/tmp/delta/people10m")
For timestamp_string
, only date or timestamp strings are accepted. For example, "2019-01-01"
and "2019-01-01T00:00:00.000Z"
.
A common pattern is to use the latest state of the Delta table throughout the execution of a job to update downstream applications.
Because Delta tables auto update, a DataFrame loaded from a Delta table may return different results across invocations if the underlying data is updated. By using time travel, you can fix the data returned by the DataFrame across invocations:
history = spark.sql("DESCRIBE HISTORY delta.`/tmp/delta/people10m`")
latest_version = history.selectExpr("max(version)").collect()
df = spark.read.format("delta").option("versionAsOf", latest_version[0][0]).load("/tmp/delta/people10m")
Examples
Fix accidental deletes to a table for the user
111
:
yesterday = spark.sql("SELECT CAST(date_sub(current_date(), 1) AS STRING)").collect()[0][0]
df = spark.read.format("delta").option("timestampAsOf", yesterday).load("/tmp/delta/events")
df.where("userId = 111").write.format("delta").mode("append").save("/tmp/delta/events")
Fix accidental incorrect updates to a table:
yesterday = spark.sql("SELECT CAST(date_sub(current_date(), 1) AS STRING)").collect()[0][0]
df = spark.read.format("delta").option("timestampAsOf", yesterday).load("/tmp/delta/events")
df.createOrReplaceTempView("my_table_yesterday")
spark.sql('''
MERGE INTO delta.`/tmp/delta/events` target
USING my_table_yesterday source
ON source.userId = target.userId
WHEN MATCHED THEN UPDATE SET *
''')
Query the number of new customers added over the last week.
last_week = spark.sql("SELECT CAST(date_sub(current_date(), 7) AS STRING)").collect()[0][0]
df = spark.read.format("delta").option("timestampAsOf", last_week).load("/tmp/delta/events")
last_week_count = df.select("userId").distinct().count()
count = spark.read.format("delta").load("/tmp/delta/events").select("userId").distinct().count()
new_customers_count = count - last_week_count
Data retention
To time travel to a previous version, you must retain both the log and the data files for that version.
The data files backing a Delta table are never deleted automatically; data files are deleted only when you run VACUUM. VACUUM
does not delete Delta log files; log files are automatically cleaned up after checkpoints are written.
By default you can time travel to a Delta table up to 30 days old unless you have:
Run
VACUUM
on your Delta table.Changed the data or log file retention periods using the following table properties:
delta.logRetentionDuration = "interval <interval>"
: controls how long the history for a table is kept. The default isinterval 30 days
.Each time a checkpoint is written, Delta automatically cleans up log entries older than the retention interval. If you set this config to a large enough value, many log entries are retained. This should not impact performance as operations against the log are constant time. Operations on history are parallel but will become more expensive as the log size increases.
delta.deletedFileRetentionDuration = "interval <interval>"
: controls how long ago a file must have been deleted before being a candidate forVACUUM
. The default isinterval 7 days
.To access 30 days of historical data even if you run
VACUUM
on the Delta table, setdelta.deletedFileRetentionDuration = "interval 30 days"
. This setting may cause your storage costs to go up.
Note
Due to log entry cleanup, instances can arise where you cannot time travel to a version that is less than the retention interval. Delta Lake requires all consecutive log entries since the previous checkpoint to time travel to a particular version. For example, with a table initially consisting of log entries for versions [0, 19] and a checkpoint at verison 10, if the log entry for version 0 is cleaned up, then you cannot time travel to versions [1, 9]. Increasing the table property delta.logRetentionDuration
can help avoid these situations.
Write to a table
Append
To atomically add new data to an existing Delta table, use append
mode:
INSERT INTO default.people10m SELECT * FROM morePeople
df.write.format("delta").mode("append").save("/tmp/delta/people10m")
df.write.format("delta").mode("append").saveAsTable("default.people10m")
df.write.format("delta").mode("append").save("/tmp/delta/people10m")
df.write.format("delta").mode("append").saveAsTable("default.people10m")
import io.delta.implicits._
df.write.mode("append").delta("/tmp/delta/people10m")
Overwrite
To atomically replace all the data in a table, use overwrite
mode:
INSERT OVERWRITE TABLE default.people10m SELECT * FROM morePeople
df.write.format("delta").mode("overwrite").save("/tmp/delta/people10m")
df.write.format("delta").mode("overwrite").saveAsTable("default.people10m")
df.write.format("delta").mode("overwrite").save("/tmp/delta/people10m")
df.write.format("delta").mode("overwrite").saveAsTable("default.people10m")
import io.delta.implicits._
df.write.mode("overwrite").delta("/tmp/delta/people10m")
Using DataFrames, you can also selectively overwrite only the data that matches an arbitrary expression. This feature is available in Delta Lake 1.1.0 and above. The following command atomically replaces events in January in the target table, which is partitioned by start_date
, with the data in df
:
df.write \
.format("delta") \
.mode("overwrite") \
.option("replaceWhere", "start_date >= '2017-01-01' AND end_date <= '2017-01-31'") \
.save("/tmp/delta/events")
df.write
.format("delta")
.mode("overwrite")
.option("replaceWhere", "start_date >= '2017-01-01' AND end_date <= '2017-01-31'")
.save("/tmp/delta/events")
This sample code writes out the data in df
, validates that it all matches the predicate, and performs an atomic replacement. If you want to write out data that doesn’t all match the predicate, to replace the matching rows in the target table, you can disable the constraint check by setting spark.databricks.delta.replaceWhere.constraintCheck.enabled
to false:
spark.conf.set("spark.databricks.delta.replaceWhere.constraintCheck.enabled", False)
spark.conf.set("spark.databricks.delta.replaceWhere.constraintCheck.enabled", false)
In Delta Lake 1.0.0 and below, replaceWhere
overwrites data matching a predicate over partition columns only. The following command atomically replaces the month in January in the target table, which is partitioned by date
, with the data in df
:
df.write \
.format("delta") \
.mode("overwrite") \
.option("replaceWhere", "birthDate >= '2017-01-01' AND birthDate <= '2017-01-31'") \
.save("/tmp/delta/people10m")
df.write
.format("delta")
.mode("overwrite")
.option("replaceWhere", "birthDate >= '2017-01-01' AND birthDate <= '2017-01-31'")
.save("/tmp/delta/people10m")
In Delta Lake 1.1.0 and above, if you want to fall back to the old behavior, you can disable the spark.databricks.delta.replaceWhere.dataColumns.enabled
flag:
spark.conf.set("spark.databricks.delta.replaceWhere.dataColumns.enabled", False)
spark.conf.set("spark.databricks.delta.replaceWhere.dataColumns.enabled", false)
Dynamic Partition Overwrites
Delta Lake 2.0 and above supports dynamic partition overwrite mode for partitioned tables.
When in dynamic partition overwrite mode, we overwrite all existing data in each logical partition for which the write will commit new data. Any existing logical partitions for which the write does not contain data will remain unchanged. This mode is only applicable when data is being written in overwrite mode: either INSERT OVERWRITE
in SQL, or a DataFrame write with df.write.mode("overwrite")
.
Configure dynamic partition overwrite mode by setting the Spark session configuration spark.sql.sources.partitionOverwriteMode
to dynamic
. You can also enable this by setting the DataFrameWriter
option partitionOverwriteMode
to dynamic
. If present, the query-specific option overrides the mode defined in the session configuration. The default for partitionOverwriteMode
is static
.
SET spark.sql.sources.partitionOverwriteMode=dynamic;
INSERT OVERWRITE TABLE default.people10m SELECT * FROM morePeople;
df.write \
.format("delta") \
.mode("overwrite") \
.option("partitionOverwriteMode", "dynamic") \
.saveAsTable("default.people10m")
df.write
.format("delta")
.mode("overwrite")
.option("partitionOverwriteMode", "dynamic")
.saveAsTable("default.people10m")
Note
Dynamic partition overwrite conflicts with the option replaceWhere
for partitioned tables.
If dynamic partition overwrite is enabled in the Spark session configuration, and
replaceWhere
is provided as aDataFrameWriter
option, data will be overwritten according to thereplaceWhere
expression (query-specific options override session configurations).If both dynamic partition overwrite and
replaceWhere
are enabled in theDataFrameWriter
options, an error will be thrown.
Important
Validate that the data being written with dynamic partition overwrite touches only the expected partitions. A single row in the incorrect partition can lead to unintentionally overwritting an entire partition. We strongly recommend using replaceWhere
to explicitly specify which data to overwrite.
If a partition has been accidentally overwritten, you can use Restore a Delta table to an earlier state to undo the change.
For Delta Lake support for updating tables, see Table deletes, updates, and merges.
Limit rows written in a file
You can use the SQL session configuration spark.sql.files.maxRecordsPerFile
to specify the maximum number of records to write to a single file for a Delta Lake table. Specifying a value of zero or a negative value represents no limit.
You can also use the DataFrameWriter option maxRecordsPerFile
when using the DataFrame APIs to write to a Delta Lake table. When maxRecordsPerFile
is specified, the value of the SQL session configuration spark.sql.files.maxRecordsPerFile
is ignored.
df.write.format("delta") \
.mode("append") \
.option("maxRecordsPerFile", "10000") \
.save("/tmp/delta/people10m")
df.write.format("delta")
.mode("append")
.option("maxRecordsPerFile", "10000")
.save("/tmp/delta/people10m")
Idempotent writes
Sometimes a job that writes data to a Delta table is restarted due to various reasons (for example, job encounters a failure). The failed job may or may not have written the data to Delta table before terminating. In the case where the data is written to the Delta table, the restarted job writes the same data to the Delta table which results in duplicate data.
To address this, Delta tables support the following DataFrameWriter
options to make the writes idempotent:
txnAppId
: A unique string that you can pass on eachDataFrame
write. For example, this can be the name of the job.txnVersion
: A monotonically increasing number that acts as transaction version. This number needs to be unique for data that is being written to the Delta table(s). For example, this can be the epoch seconds of the instant when the query is attempted for the first time. Any subsequent restarts of the same job needs to have the same value fortxnVersion
.
The above combination of options needs to be unique for each new data that is being ingested into the Delta table and the txnVersion
needs to be higher than the last data that was ingested into the Delta table. For example:
Last successfully written data contains option values as
dailyETL:23423
(txnAppId:txnVersion
).Next write of data should have
txnAppId = dailyETL
andtxnVersion
as at least23424
(one more than the last written datatxnVersion
).Any attempt to write data with
txnAppId = dailyETL
andtxnVersion
as23422
or less is ignored because thetxnVersion
is less than the last recordedtxnVersion
in the table.Attempt to write data with
txnAppId:txnVersion
asanotherETL:23424
is successful writing data to the table as it contains a differenttxnAppId
compared to the same option value in last ingested data.
Warning
This solution assumes that the data being written to Delta table(s) in multiple retries of the job is same. If a write attempt in a Delta table succeeds but due to some downstream failure there is a second write attempt with same txn options but different data, then that second write attempt will be ignored. This can cause unexpected results.
Example
app_id = ... # A unique string that is used as an application ID.
version = ... # A monotonically increasing number that acts as transaction version.
dataFrame.write.format(...).option("txnVersion", version).option("txnAppId", app_id).save(...)
val appId = ... // A unique string that is used as an application ID.
version = ... // A monotonically increasing number that acts as transaction version.
dataFrame.write.format(...).option("txnVersion", version).option("txnAppId", appId).save(...)
Set user-defined commit metadata
You can specify user-defined strings as metadata in commits made by these operations, either using the DataFrameWriter option userMetadata
or the SparkSession configuration spark.databricks.delta.commitInfo.userMetadata
. If both of them have been specified, then the option takes preference. This user-defined metadata is readable in the history operation.
SET spark.databricks.delta.commitInfo.userMetadata=overwritten-for-fixing-incorrect-data
INSERT OVERWRITE default.people10m SELECT * FROM morePeople
df.write.format("delta") \
.mode("overwrite") \
.option("userMetadata", "overwritten-for-fixing-incorrect-data") \
.save("/tmp/delta/people10m")
df.write.format("delta")
.mode("overwrite")
.option("userMetadata", "overwritten-for-fixing-incorrect-data")
.save("/tmp/delta/people10m")
Schema validation
Delta Lake automatically validates that the schema of the DataFrame being written is compatible with the schema of the table. Delta Lake uses the following rules to determine whether a write from a DataFrame to a table is compatible:
All DataFrame columns must exist in the target table. If there are columns in the DataFrame not present in the table, an exception is raised. Columns present in the table but not in the DataFrame are set to null.
DataFrame column data types must match the column data types in the target table. If they don’t match, an exception is raised.
DataFrame column names cannot differ only by case. This means that you cannot have columns such as “Foo” and “foo” defined in the same table. While you can use Spark in case sensitive or insensitive (default) mode, Parquet is case sensitive when storing and returning column information. Delta Lake is case-preserving but insensitive when storing the schema and has this restriction to avoid potential mistakes, data corruption, or loss issues.
Delta Lake support DDL to add new columns explicitly and the ability to update schema automatically.
If you specify other options, such as partitionBy
, in combination with append mode, Delta Lake validates that they match and throws an error for any mismatch. When partitionBy
is not present, appends automatically follow the partitioning of the existing data.
Update 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
You can make these changes explicitly using DDL or implicitly using DML.
Important
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.
Explicitly update schema
You can use the following DDL to explicitly change the schema of a table.
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], ...)
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.
Change column comment or ordering
ALTER TABLE table_name ALTER [COLUMN] col_name col_name data_type [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 nested_col_name data_type [COMMENT col_comment] [FIRST|AFTER colA_name]
Replace columns
ALTER TABLE table_name REPLACE COLUMNS (col_name1 col_type1 [COMMENT col_comment1], ...)
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
Rename columns
Note
This feature is available in Delta Lake 1.2.0 and above. This feature is currently experimental.
To rename columns without rewriting any of the columns’ existing data, you must enable column mapping for the table. See enable 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
Drop columns
Note
This feature is available in Delta Lake 2.0 and above. This feature is currently experimental.
To drop columns as a metadata-only operation without rewriting any data files, you must enable column mapping for the table. See enable column mapping.
Important
Dropping a column from metadata does not delete the underlying data for the column in files.
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)
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:
Automatic schema update
Delta Lake can automatically update the schema of a table as part of a DML transaction (either appending or overwriting), and make the schema compatible with the data being written.
Add columns
Columns that are present in the DataFrame but missing from the table are automatically added as part of a write transaction when:
write
orwriteStream
have.option("mergeSchema", "true")
spark.databricks.delta.schema.autoMerge.enabled
istrue
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.
NullType
columns
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")
Views on tables
Delta Lake supports the creation of views on top of Delta tables just like you might with a data source table.
The core challenge when you operate with views is resolving the schemas. If you alter a Delta table schema, you must recreate derivative views to account for any additions to the schema. For instance, if you add a new column to a Delta table, you must make sure that this column is available in the appropriate views built on top of that base table.
Table properties
You can store your own metadata as a table property using TBLPROPERTIES
in CREATE
and ALTER
. You can then SHOW
that metadata. For example:
ALTER TABLE default.people10m SET TBLPROPERTIES ('department' = 'accounting', 'delta.appendOnly' = 'true');
-- Show the table's properties.
SHOW TBLPROPERTIES default.people10m;
-- Show just the 'department' table property.
SHOW TBLPROPERTIES default.people10m ('department');
TBLPROPERTIES
are stored as part of Delta table metadata. You cannot define new TBLPROPERTIES
in a CREATE
statement if a Delta table already exists in a given location.
In addition, to tailor behavior and performance, Delta Lake supports certain Delta table properties:
Block deletes and updates in a Delta table:
delta.appendOnly=true
.Configure the time travel retention properties:
delta.logRetentionDuration=<interval-string>
anddelta.deletedFileRetentionDuration=<interval-string>
. For details, see Data retention.Configure the number of columns for which statistics are collected:
delta.dataSkippingNumIndexedCols=n
. This property indicates to the writer that statistics are to be collected only for the firstn
columns in the table. Also the data skipping code ignores statistics for any column beyond this column index. This property takes affect only for new data that is written out.
Note
Modifying a Delta table property is a write operation that will conflict with other concurrent write operations, causing them to fail. We recommend that you modify a table property only when there are no concurrent write operations on the table.
You can also set delta.
-prefixed properties during the first commit to a Delta table using Spark configurations. For example, to initialize a Delta table with the property delta.appendOnly=true
, set the Spark configuration spark.databricks.delta.properties.defaults.appendOnly
to true
. For example:
spark.sql("SET spark.databricks.delta.properties.defaults.appendOnly = true")
spark.conf.set("spark.databricks.delta.properties.defaults.appendOnly", "true")
spark.conf.set("spark.databricks.delta.properties.defaults.appendOnly", "true")
See also the Delta table properties reference.
Table metadata
Delta Lake has rich features for exploring table metadata.
It supports DESCRIBE TABLE
.
It also provides the following unique commands:
DESCRIBE DETAIL
Provides information about schema, partitioning, table size, and so on. For details, see Retrieve Delta table details.
DESCRIBE HISTORY
Provides provenance information, including the operation, user, and so on, and operation metrics for each write to a table. Table history is retained for 30 days. For details, see Retrieve Delta table history.
Configure SparkSession
For many Delta Lake operations, you enable integration with Apache Spark DataSourceV2 and Catalog APIs (since 3.0) by setting the following configurations when you create a new SparkSession
.
from pyspark.sql import SparkSession
spark = SparkSession \
.builder \
.appName("...") \
.master("...") \
.config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension") \
.config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog") \
.getOrCreate()
import org.apache.spark.sql.SparkSession
val spark = SparkSession
.builder()
.appName("...")
.master("...")
.config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
.config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog")
.getOrCreate()
import org.apache.spark.sql.SparkSession;
SparkSession spark = SparkSession
.builder()
.appName("...")
.master("...")
.config("spark.sql.extensions", "io.delta.sql.DeltaSparkSessionExtension")
.config("spark.sql.catalog.spark_catalog", "org.apache.spark.sql.delta.catalog.DeltaCatalog")
.getOrCreate();
Alternatively, you can add configurations when submitting your Spark application using spark-submit
or when starting spark-shell
or pyspark
by specifying them as command-line parameters.
spark-submit --conf "spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension" --conf "spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog" ...
pyspark --conf "spark.sql.extensions=io.delta.sql.DeltaSparkSessionExtension" --conf "spark.sql.catalog.spark_catalog=org.apache.spark.sql.delta.catalog.DeltaCatalog"
Configure storage credentials
Delta Lake uses Hadoop FileSystem APIs to access the storage systems. The credentails for storage systems usually can be set through Hadoop configurations. Delta Lake provides multiple ways to set Hadoop configurations similar to Apache Spark.
Spark configurations
When you start a Spark application on a cluster, you can set the Spark configurations in the form of spark.hadoop.*
to pass your custom Hadoop configurations. For example, Setting a value for spark.hadoop.a.b.c
will pass the value as a Hadoop configuration a.b.c
, and Delta Lake will use it to access Hadoop FileSystem APIs.
See Spark doc for more details.
SQL session configurations
Spark SQL will pass all of the current SQL session configurations to Delta Lake, and Delta Lake will use them to access Hadoop FileSystem APIs. For example, SET a.b.c=x.y.z
will tell Delta Lake to pass the value x.y.z
as a Hadoop configuration a.b.c
, and Delta Lake will use it to access Hadoop FileSystem APIs.
DataFrame options
Besides setting Hadoop file system configurations through the Spark (cluster) configurations or SQL session configurations, Delta supports reading Hadoop file system configurations from DataFrameReader
and DataFrameWriter
options (that is, option keys that start with the fs.
prefix) when the table is read or written, by using DataFrameReader.load(path)
or DataFrameWriter.save(path)
.
For example, you can pass your storage credentails through DataFrame options:
df1 = spark.read.format("delta") \
.option("fs.azure.account.key.<storage-account-name>.dfs.core.windows.net", "<storage-account-access-key-1>") \
.read("...")
df2 = spark.read.format("delta") \
.option("fs.azure.account.key.<storage-account-name>.dfs.core.windows.net", "<storage-account-access-key-2>") \
.read("...")
df1.union(df2).write.format("delta") \
.mode("overwrite") \
.option("fs.azure.account.key.<storage-account-name>.dfs.core.windows.net", "<storage-account-access-key-3>") \
.save("...")
val df1 = spark.read.format("delta")
.option("fs.azure.account.key.<storage-account-name>.dfs.core.windows.net", "<storage-account-access-key-1>")
.read("...")
val df2 = spark.read.format("delta")
.option("fs.azure.account.key.<storage-account-name>.dfs.core.windows.net", "<storage-account-access-key-2>")
.read("...")
df1.union(df2).write.format("delta")
.mode("overwrite")
.option("fs.azure.account.key.<storage-account-name>.dfs.core.windows.net", "<storage-account-access-key-3>")
.save("...")
You can find the details of the Hadoop file system configurations for your storage in Storage configuration.