Delta Live Tables Python language reference
This article provides details for the Delta Live Tables Python programming interface.
For information on the SQL API, see the Delta Live Tables SQL language reference.
For details specific to configuring Auto Loader, see What is Auto Loader?.
Limitations
The Delta Live Tables Python interface has the following limitations:
- The Python
table
andview
functions must return a DataFrame. Some functions that operate on DataFrames do not return DataFrames and should not be used. Because DataFrame transformations are executed after the full dataflow graph has been resolved, using such operations might have unintended side effects. These operations include functions such ascollect()
,count()
,toPandas()
,save()
, andsaveAsTable()
. However, you can include these functions outside oftable
orview
function definitions because this code is run once during the graph initialization phase. - The
pivot()
function is not supported. Thepivot
operation in Spark requires eager loading of input data to compute the schema of the output. This capability is not supported in Delta Live Tables.
Import the dlt
Python module
Delta Live Tables Python functions are defined in the dlt
module. Your pipelines implemented with the Python API must import this module:
import dlt
Create a Delta Live Tables materialized view or streaming table
In Python, Delta Live Tables determines whether to update a dataset as a materialized view or streaming table based on the defining query. The @table
decorator is used to define both materialized views and streaming tables.
To define a materialized view in Python, apply @table
to a query that performs a static read against a data source. To define a streaming table, apply @table
to a query that performs a streaming read against a data source. Both dataset types have the same syntax specification as follows:
import dlt
@dlt.table(
name="<name>",
comment="<comment>",
spark_conf={"<key>" : "<value", "<key" : "<value>"},
table_properties={"<key>" : "<value>", "<key>" : "<value>"},
path="<storage-location-path>",
partition_cols=["<partition-column>", "<partition-column>"],
schema="schema-definition",
temporary=False)
@dlt.expect
@dlt.expect_or_fail
@dlt.expect_or_drop
@dlt.expect_all
@dlt.expect_all_or_drop
@dlt.expect_all_or_fail
def <function-name>():
return (<query>)
Create a Delta Live Tables view
To define a view in Python, apply the @view
decorator. Like the @table
decorator, you can use views in Delta Live Tables for either static or streaming datasets. The following is the syntax for defining views with Python:
import dlt
@dlt.view(
name="<name>",
comment="<comment>")
@dlt.expect
@dlt.expect_or_fail
@dlt.expect_or_drop
@dlt.expect_all
@dlt.expect_all_or_drop
@dlt.expect_all_or_fail
def <function-name>():
return (<query>)
Example: Define tables and views
To define a table or view in Python, apply the @dlt.view
or @dlt.table
decorator to a function. You can use the function name or the name
parameter to assign the table or view name. The following example defines two different datasets: a view called taxi_raw
that takes a JSON file as the input source and a table called filtered_data
that takes the taxi_raw
view as input:
import dlt
@dlt.view
def taxi_raw():
return spark.read.format("json").load("/databricks-datasets/nyctaxi/sample/json/")
# Use the function name as the table name
@dlt.table
def filtered_data():
return dlt.read("taxi_raw").where(...)
# Use the name parameter as the table name
@dlt.table(
name="filtered_data")
def create_filtered_data():
return dlt.read("taxi_raw").where(...)
Example: Access a dataset defined in the same pipeline
In addition to reading from external data sources, you can access datasets defined in the same pipeline with the Delta Live Tables read()
function. The following example demonstrates creating a customers_filtered
dataset using the read()
function:
@dlt.table
def customers_raw():
return spark.read.format("csv").load("/data/customers.csv")
@dlt.table
def customers_filteredA():
return dlt.read("customers_raw").where(...)
You can also use the spark.table()
function to access a dataset defined in the same pipeline. When using the spark.table()
function to access a dataset defined in the pipeline, in the function argument prepend the LIVE
keyword to the dataset name:
@dlt.table
def customers_raw():
return spark.read.format("csv").load("/data/customers.csv")
@dlt.table
def customers_filteredB():
return spark.table("LIVE.customers_raw").where(...)
Example: Read from a table registered in a metastore
To read data from a table registered in the Hive metastore, in the function argument omit the LIVE
keyword and optionally qualify the table name with the database name:
@dlt.table
def customers():
return spark.table("sales.customers").where(...)
For an example of reading from a Unity Catalog table, see Ingest data into a Unity Catalog pipeline.
Example: Access a dataset using spark.sql
You can also return a dataset using a spark.sql
expression in a query function. To read from an internal dataset, prepend LIVE.
to the dataset name:
@dlt.table
def chicago_customers():
return spark.sql("SELECT * FROM LIVE.customers_cleaned WHERE city = 'Chicago'")
Control how tables are materialized
Tables also offer additional control of their materialization:
- Specify how tables are partitioned using
partition_cols
. You can use partitioning to speed up queries. - You can set table properties when you define a view or table. See Delta Live Tables table properties.
- Set a storage location for table data using the
path
setting. By default, table data is stored in the pipeline storage location ifpath
isn’t set. - You can use generated columns in your schema definition. See Example: Specify a schema and partition columns.
Note
For tables less than 1 TB in size, Databricks recommends letting Delta Live Tables control data organization. Unless you expect your table to grow beyond a terabyte, you should generally not specify partition columns.
Example: Specify a schema and partition columns
You can optionally specify a table schema using a Python StructType
or a SQL DDL string. When specified with a DDL string, the definition can include generated columns.
The following examples creates a table called sales
with a schema specified using a Python StructType
:
sales_schema = StructType([
StructField("customer_id", StringType(), True),
StructField("customer_name", StringType(), True),
StructField("number_of_line_items", StringType(), True),
StructField("order_datetime", StringType(), True),
StructField("order_number", LongType(), True)]
)
@dlt.table(
comment="Raw data on sales",
schema=sales_schema)
def sales():
return ("...")
The following example specifies the schema for a table using a DDL string, defines a generated column, and defines a partition column:
@dlt.table(
comment="Raw data on sales",
schema="""
customer_id STRING,
customer_name STRING,
number_of_line_items STRING,
order_datetime STRING,
order_number LONG,
order_day_of_week STRING GENERATED ALWAYS AS (dayofweek(order_datetime))
""",
partition_cols = ["order_day_of_week"])
def sales():
return ("...")
By default, Delta Live Tables infers the schema from the table
definition if you don’t specify a schema.
Configure a streaming table to ignore changes in a source streaming table
Note
- To use the
skipChangeCommits
flag, you must select the Preview channel in your pipeline settings. - The
skipChangeCommits
flag works only withspark.readStream
using theoption()
function. You cannot use this flag in adlt.read_stream()
function.
By default, streaming tables require append-only sources. When a streaming table uses another streaming table as a source, and the source streaming table requires updates or deletes, for example, GDPR “right to be forgotten” processing, the skipChangeCommits
flag can be set on the target streaming table to ignore those changes. For more information about this flag, see Ignore updates and deletes.
@table
def b():
return spark.readStream.option("skipChangeCommits", "true").table("LIVE.A")
Python Delta Live Tables properties
The following tables describe the options and properties you can specify while defining tables and views with Delta Live Tables:
@table or @view |
---|
name Type: str An optional name for the table or view. If not defined, the function name is used as the table or view name. |
comment Type: str An optional description for the table. |
spark_conf Type: dict An optional list of Spark configurations for the execution of this query. |
table_properties Type: dict An optional list of table properties for the table. |
path Type: str An optional storage location for table data. If not set, the system will default to the pipeline storage location. |
partition_cols Type: a collection of str An optional collection, for example, a list , of one or more columns to use for partitioning the table. |
schema Type: str or StructType An optional schema definition for the table. Schemas can be defined as a SQL DDL string, or with a Python StructType . |
temporary Type: bool Create a temporary table. No metadata is persisted for this table. The default is ‘False’. |
Table or view definition |
---|
def () A Python function that defines the dataset. If the name parameter is not set, then <function-name> is used as the target dataset name. |
query A Spark SQL statement that returns a Spark Dataset or Koalas DataFrame. Use dlt.read() or spark.table() to perform a complete read from a dataset defined in the same pipeline. When using the spark.table() function to read from a dataset defined in the same pipeline, prepend the LIVE keyword to the dataset name in the function argument. For example, to read from a dataset named customers :spark.table("LIVE.customers") You can also use the spark.table() function to read from a table registered in the metastore by omitting the LIVE keyword and optionally qualifying the table name with the database name:spark.table("sales.customers") Use dlt.read_stream() to perform a streaming read from a dataset defined in the same pipeline.Use the spark.sql function to define a SQL query to create the return dataset.Use PySpark syntax to define Delta Live Tables queries with Python. |
Expectations |
---|
@expect(“description”, “constraint”) Declare a data quality constraint identified by description . If a row violates the expectation, include the row in the target dataset. |
@expect_or_drop(“description”, “constraint”) Declare a data quality constraint identified by description . If a row violates the expectation, drop the row from the target dataset. |
@expect_or_fail(“description”, “constraint”) Declare a data quality constraint identified by description . If a row violates the expectation, immediately stop execution. |
@expect_all(expectations) Declare one or more data quality constraints. expectations is a Python dictionary, where the key is the expectation description and the value is the expectation constraint. If a row violates any of the expectations, include the row in the target dataset. |
@expect_all_or_drop(expectations) Declare one or more data quality constraints. expectations is a Python dictionary, where the key is the expectation description and the value is the expectation constraint. If a row violates any of the expectations, drop the row from the target dataset. |
@expect_all_or_fail(expectations) Declare one or more data quality constraints. expectations is a Python dictionary, where the key is the expectation description and the value is the expectation constraint. If a row violates any of the expectations, immediately stop execution. |
Change data capture with Python in Delta Live Tables
Important
Delta Live Tables support for SCD type 2 is in Public Preview.
Use the apply_changes()
function in the Python API to use Delta Live Tables CDC functionality. The Delta Live Tables Python CDC interface also provides the create_streaming_table() function. You can use this function to create the target table required by the apply_changes()
function.
apply_changes(
target = "<target-table>",
source = "<data-source>",
keys = ["key1", "key2", "keyN"],
sequence_by = "<sequence-column>",
ignore_null_updates = False,
apply_as_deletes = None,
apply_as_truncates = None,
column_list = None,
except_column_list = None,
stored_as_scd_type = <type>,
track_history_column_list = None,
track_history_except_column_list = None
)
Note
The default behavior for INSERT
and UPDATE
events is to upsert CDC events from the source: update any rows in the target table that match the specified key(s) or insert a new row when a matching record does not exist in the target table. Handling for DELETE
events can be specified with the APPLY AS DELETE WHEN
condition.
Important
You must declare a target streaming table to apply changes into. You can optionally specify the schema for your target table. When specifying the schema of the apply_changes
target table, you must also include the __START_AT
and __END_AT
columns with the same data type as the sequence_by
field.
See Change data capture with Delta Live Tables.
Arguments |
---|
target Type: str The name of the table to be updated. You can use the create_streaming_table() function to create the target table before executing the apply_changes() function.This parameter is required. |
source Type: str The data source containing CDC records. This parameter is required. |
keys Type: list The column or combination of columns that uniquely identify a row in the source data. This is used to identify which CDC events apply to specific records in the target table. You can specify either: * A list of strings: ["userId", "orderId"] * A list of Spark SQL col() functions: [col("userId"), col("orderId"] Arguments to col() functions cannot include qualifiers. For example, you can use col(userId) , but you cannot use col(source.userId) .This parameter is required. |
sequence_by Type: str or col() The column name specifying the logical order of CDC events in the source data. Delta Live Tables uses this sequencing to handle change events that arrive out of order. You can specify either: * A string: "sequenceNum" * A Spark SQL col() function: col("sequenceNum") Arguments to col() functions cannot include qualifiers. For example, you can use col(userId) , but you cannot use col(source.userId) .This parameter is required. |
ignore_null_updates Type: bool Allow ingesting updates containing a subset of the target columns. When a CDC event matches an existing row and ignore_null_updates is True , columns with a null will retain their existing values in the target. This also applies to nested columns with a value of null . When ignore_null_updates is False , existing values will be overwritten with null values.This parameter is optional. The default is False . |
apply_as_deletes Type: str or expr() Specifies when a CDC event should be treated as a DELETE rather than an upsert. To handle out-of-order data, the deleted row is temporarily retained as a tombstone in the underlying Delta table, and a view is created in the metastore that filters out these tombstones. The retention interval can be configured with thepipelines.cdc.tombstoneGCThresholdInSeconds table property.You can specify either: * A string: "Operation = 'DELETE'" * A Spark SQL expr() function: expr("Operation = 'DELETE'") This parameter is optional. |
apply_as_truncates Type: str or expr() Specifies when a CDC event should be treated as a full table TRUNCATE . Because this clause triggers a full truncate of the target table, it should be used only for specific use cases requiring this functionality.The apply_as_truncates parameter is supported only for SCD type 1. SCD type 2 does not support truncate.You can specify either: * A string: "Operation = 'TRUNCATE'" * A Spark SQL expr() function: expr("Operation = 'TRUNCATE'") This parameter is optional. |
column_list except_column_list Type: list A subset of columns to include in the target table. Use column_list to specify the complete list of columns to include. Use except_column_list to specify the columns to exclude. You can declare either value as a list of strings or as Spark SQL col() functions:* column_list = ["userId", "name", "city"] .* column_list = [col("userId"), col("name"), col("city")] * except_column_list = ["operation", "sequenceNum"] * except_column_list = [col("operation"), col("sequenceNum") Arguments to col() functions cannot include qualifiers. For example, you can use col(userId) , but you cannot use col(source.userId) .This parameter is optional. The default is to include all columns in the target table when no column_list or except_column_list argument is passed to the function. |
stored_as_scd_type Type: str or int Whether to store records as SCD type 1 or SCD type 2. Set to 1 for SCD type 1 or 2 for SCD type 2.This clause is optional. The default is SCD type 1. |
track_history_column_list track_history_except_column_list Type: list A subset of output columns to be tracked for history in the target table. When pipelines.enableTrackHistory is set, use track_history_column_list to specify the complete list of columns to be tracked. Use track_history_except_column_list to specify the columns to be excluded from tracking. You can declare either value as a list of strings or as Spark SQL col() functions: - track_history_column_list = ["userId", "name", "city"] . - track_history_column_list = [col("userId"), col("name"), col("city")] - track_history_except_column_list = ["operation", "sequenceNum"] - track_history_except_column_list = [col("operation"), col("sequenceNum") Arguments to col() functions cannot include qualifiers. For example, you can use col(userId) , but you cannot use col(source.userId) .This parameter is optional. The default is to include all columns in the target table when no track_history_column_list ortrack_history_except_column_list argument is passed to the function.To use these parameters, you must set pipelines.enableTrackHistory in the pipeline settings. Otherwise, an exception is thrown. When pipelines.enableTrackHistory is not set, a history record is generated for every input row. |
Create a target table for CDC output
Use the create_streaming_table()
function to create a target table for the apply_changes()
output records.
Note
The create_target_table()
and create_streaming_live_table()
functions are deprecated. Databricks recommends updating existing code to use the create_streaming_table()
function.
create_streaming_table(
name = "<table-name>",
comment = "<comment>"
spark_conf={"<key>" : "<value", "<key" : "<value>"},
table_properties={"<key>" : "<value>", "<key>" : "<value>"},
partition_cols=["<partition-column>", "<partition-column>"],
path="<storage-location-path>",
schema="schema-definition"
)
Arguments |
---|
name Type: str The table name. This parameter is required. |
comment Type: str An optional description for the table. |
spark_conf Type: dict An optional list of Spark configurations for the execution of this query. |
table_properties Type: dict An optional list of table properties for the table. |
partition_cols Type: array An optional list of one or more columns to use for partitioning the table. |
path Type: str An optional storage location for table data. If not set, the system will default to the pipeline storage location. |
schema Type: str or StructType An optional schema definition for the table. Schemas can be defined as a SQL DDL string, or with a Python StructType . |
Feedback
Feedback senden und anzeigen für