Skip to content
You're viewing the beta version. Looking for legacy docs? Click here.

Storage configuration

Delta Lake ACID guarantees are predicated on the atomicity and durability guarantees of the storage system. Specifically, Delta Lake relies on the following when interacting with storage systems:

  • Atomic visibility: There must a way for a file to visible in its entirety or not visible at all.
  • Mutual exclusion: Only one writer must be able to create (or rename) a file at the final destination.
  • Consistent listing: Once a file has been written in a directory, all future listings for that directory must return that file.

Because storage systems do not necessarily provide all of these guarantees out-of-the-box, Delta Lake transactional operations typically go through the LogStore API instead of accessing the storage system directly. To provide the ACID guarantees for different storage systems, you may have to use different LogStore implementations. This article covers how to configure Delta Lake for various storage systems. There are two categories of storage systems:

  • Storage systems with built-in support: For some storage systems, you do not need additional configurations. Delta Lake uses the scheme of the path (that is, s3a in s3a://path) to dynamically identify the storage system and use the corresponding LogStore implementation that provides the transactional guarantees. However, for S3, there are additional caveats on concurrent writes. See the section on S3 for details.

  • Other storage systems: The LogStore, similar to Apache Spark, uses Hadoop FileSystem API to perform reads and writes. So Delta Lake supports concurrent reads on any storage system that provides an implementation of FileSystem API. For concurrent writes with transactional guarantees, there are two cases based on the guarantees provided by FileSystem implementation. If the implementation provides consistent listing and atomic renames-without-overwrite (that is, rename(..., overwrite = false) will either generate the target file atomically or fail if it already exists with java.nio.file.FileAlreadyExistsException), then the default LogStore implementation using renames will allow concurrent writes with guarantees. Otherwise, you must configure a custom implementation of LogStore by setting the following Spark configuration:

spark.delta.logStore.<scheme>.impl=<full-qualified-class-name>

where <scheme> is the scheme of the paths of your storage system. This configures Delta Lake to dynamically use the given LogStore implementation only for those paths. You can have multiple such configurations for different schemes in your application, thus allowing it to simultaneously read and write from different storage systems.

Delta Lake supports reads and writes to S3 in two different modes: Single-cluster and Multi-cluster.

Single-clusterMulti-cluster
ConfigurationComes with Delta Lake out-of-the-boxIs experimental and requires extra configuration
ReadsSupports concurrent reads from multiple clustersSupports concurrent reads from multiple clusters
WritesSupports concurrent writes from a single Spark driverSupports multi-cluster writes
PermissionsS3 credentialsS3 and DynamoDB operating permissions

In this default mode, Delta Lake supports concurrent reads from multiple clusters, but concurrent writes to S3 must originate from a single Spark driver in order for Delta Lake to provide transactional guarantees. This is because S3 currently does not provide mutual exclusion, that is, there is no way to ensure that only one writer is able to create a file.

  • S3 credentials: IAM roles (recommended) or access keys
  • Apache Spark associated with the corresponding Delta Lake version.
  • Hadoop’s AWS connector (hadoop-aws) for the version of Hadoop that Apache Spark is compiled for.

This section explains how to quickly start reading and writing Delta tables on S3 using single-cluster mode. For a detailed explanation of the configuration, see Setup Configuration (S3 multi-cluster).

  1. Use the following command to launch a Spark shell with Delta Lake and S3 support (assuming you use Spark 3.5.3 which is pre-built for Hadoop 3.3.4):
Terminal window
bin/spark-shell \
--packages io.delta:delta-spark_2.12:3.3.0,org.apache.hadoop:hadoop-aws:3.3.4 \
--conf spark.hadoop.fs.s3a.access.key=<your-s3-access-key> \
--conf spark.hadoop.fs.s3a.secret.key=<your-s3-secret-key>
  1. Try out some basic Delta table operations on S3 (in Scala):
// Create a Delta table on S3:
spark.range(5).write.format("delta").save("s3a://<your-s3-bucket>/<path-to-delta-table>")
// Read a Delta table on S3:
spark.read.format("delta").load("s3a://<your-s3-bucket>/<path-to-delta-table>").show()

For other languages and more examples of Delta table operations, see the Quickstart page.

For efficient listing of Delta Lake metadata files on S3, set the configuration delta.enableFastS3AListFrom=true. This performance optimization is in experimental support mode. It will only work on S3A filesystems and will not work on Amazon’s EMR default filesystem S3.

bin/spark-shell \
--packages io.delta:delta-spark_2.12:3.3.0,org.apache.hadoop:hadoop-aws:3.3.4 \
--conf spark.hadoop.fs.s3a.access.key=<your-s3-access-key> \
--conf spark.hadoop.fs.s3a.secret.key=<your-s3-secret-key> \
--conf "spark.hadoop.delta.enableFastS3AListFrom=true

This mode supports concurrent writes to S3 from multiple clusters and has to be explicitly enabled by configuring Delta Lake to use the right LogStore implementation. This implementation uses DynamoDB to provide the mutual exclusion that S3 is lacking.

This section explains how to quickly start reading and writing Delta tables on S3 using multi-cluster mode.

  1. Use the following command to launch a Spark shell with Delta Lake and S3 support (assuming you use Spark 3.5.3 which is pre-built for Hadoop 3.3.4):
Terminal window
bin/spark-shell \
--packages io.delta:delta-spark_2.12:3,org.apache.hadoop:hadoop-aws:3.3.4,io.delta:delta-storage-s3-dynamodb:3.3.0 \
--conf spark.hadoop.fs.s3a.access.key=<your-s3-access-key> \
--conf spark.hadoop.fs.s3a.secret.key=<your-s3-secret-key> \
--conf spark.delta.logStore.s3a.impl=io.delta.storage.S3DynamoDBLogStore \
--conf spark.io.delta.storage.S3DynamoDBLogStore.ddb.region=us-west-2
  1. Try out some basic Delta table operations on S3 (in Scala):
// Create a Delta table on S3:
spark.range(5).write.format("delta").save("s3a://<your-s3-bucket>/<path-to-delta-table>")
// Read a Delta table on S3:
spark.read.format("delta").load("s3a://<your-s3-bucket>/<path-to-delta-table>").show()

For other languages and more examples of Delta table operations, see the Quickstart page.

  1. Create the DynamoDB table.

    You have the choice of creating the DynamoDB table yourself (recommended) or having it created for you automatically.

    • Creating the DynamoDB table yourself

      This DynamoDB table will maintain commit metadata for multiple Delta tables, and it is important that it is configured with the Read/Write Capacity Mode (for example, on-demand or provisioned) that is right for your use cases. As such, we strongly recommend that you create your DynamoDB table yourself. The following example uses the AWS CLI. To learn more, see the create-table command reference.

Terminal window
aws dynamodb create-table \
--region us-east-1 \
--table-name delta_log \
--attribute-definitions AttributeName=tablePath,AttributeType=S \
AttributeName=fileName,AttributeType=S \
--key-schema AttributeName=tablePath,KeyType=HASH \
AttributeName=fileName,KeyType=RANGE \
--billing-mode PAY_PER_REQUEST
  1. Follow the configuration steps listed in Configuration (S3 single-cluster) section.

  2. Include the delta-storage-s3-dynamodb JAR in the classpath.

  3. Configure the LogStore implementation in your Spark session.

    First, configure this LogStore implementation for the scheme s3. You can replicate this command for schemes s3a and s3n as well.

spark.delta.logStore.s3.impl=io.delta.storage.S3DynamoDBLogStore

Next, specify additional information necessary to instantiate the DynamoDB client. You must instantiate the DynamoDB client with the same tableName and region each Spark session for this multi-cluster mode to work correctly. A list of per-session configurations and their defaults is given below:

spark.io.delta.storage.S3DynamoDBLogStore.ddb.tableName=delta_log
spark.io.delta.storage.S3DynamoDBLogStore.ddb.region=us-east-1
spark.io.delta.storage.S3DynamoDBLogStore.credentials.provider=<AWSCredentialsProvider* used by the client>
spark.io.delta.storage.S3DynamoDBLogStore.provisionedThroughput.rcu=5
spark.io.delta.storage.S3DynamoDBLogStore.provisionedThroughput.wcu=5

Production Configuration (S3 multi-cluster)

Section titled “Production Configuration (S3 multi-cluster)”

By this point, this multi-cluster setup is fully operational. However, there is extra configuration you may do to improve performance and optimize storage when running in production.

  1. Adjust your Read and Write Capacity Mode.

    If you are using the default DynamoDB table created for you by this LogStore implementation, its default RCU and WCU might not be enough for your workloads. You can adjust the provisioned throughput or update to On-Demand Mode.

  2. Cleanup old DynamoDB entries using Time to Live (TTL).

    Once a DynamoDB metadata entry is marked as complete, and after sufficient time such that we can now rely on S3 alone to prevent accidental overwrites on its corresponding Delta file, it is safe to delete that entry from DynamoDB. The cheapest way to do this is using DynamoDB’s TTL feature which is a free, automated means to delete items from your DynamoDB table.

    Run the following command on your given DynamoDB table to enable TTL:

Terminal window
aws dynamodb update-time-to-live \
--region us-east-1 \
--table-name delta_log \
--time-to-live-specification "Enabled=true, AttributeName=expireTime"

The default expireTime will be one day after the DynamoDB entry was marked as completed.

  1. Cleanup old AWS S3 temp files using S3 Lifecycle Expiration.

    In this LogStore implementation, a temp file is created containing a copy of the metadata to be committed into the Delta log. Once that commit to the Delta log is complete, and after the corresponding DynamoDB entry has been removed, it is safe to delete this temp file. In practice, only the latest temp file will ever be used during recovery of a failed commit.

    Here are two simple options for deleting these temp files:

    1. Delete manually using S3 CLI.

      This is the safest option. The following command will delete all but the latest temp file in your given <bucket> and <table>:

Terminal window
aws s3 ls s3://<bucket>/<delta_table_path>/_delta_log/.tmp/ --recursive | awk 'NF>1{print $4}' | grep . | sort | head -n -1 | while read -r line ; do
echo "Removing ${line}"
aws s3 rm s3://<bucket>/<delta_table_path>/_delta_log/.tmp/${line}
done
  1. Delete using an S3 Lifecycle Expiration Rule

    A more automated option is to use an S3 Lifecycle Expiration rule, with filter prefix pointing to the <delta_table_path>/_delta_log/.tmp/ folder located in your table path, and an expiration value of 30 days.

There are a variety of ways to configuring a bucket lifecycle configuration, described in AWS docs here.

One way to do this is using S3’s put-bucket-lifecycle-configuration command. See S3 Lifecycle Configuration for details. An example rule and command invocation is given below:

In a file referenced as file://lifecycle.json:

{
"Rules":[
{
"ID":"expire_tmp_files",
"Filter":{
"Prefix":"path/to/table/_delta_log/.tmp/"
},
"Status":"Enabled",
"Expiration":{
"Days":30
}
}
]
}
Terminal window
aws s3api put-bucket-lifecycle-configuration \
--bucket my-bucket \
--lifecycle-configuration file://lifecycle.json

Delta Lake has built-in support for the various Azure storage systems with full transactional guarantees for concurrent reads and writes from multiple clusters.

Delta Lake relies on Hadoop FileSystem APIs to access Azure storage services. Specifically, Delta Lake requires the implementation of FileSystem.rename() to be atomic, which is only supported in newer Hadoop versions (Hadoop-15156 and Hadoop-15086). For this reason, you may need to build Spark with newer Hadoop versions and use them for deploying your application. See Specifying the Hadoop Version and Enabling YARN for building Spark with a specific Hadoop version and Quickstart for setting up Spark with Delta Lake.

Here is a list of requirements specific to each type of Azure storage system:

  • A shared key or shared access signature (SAS)
  • Delta Lake 0.2.0 or above
  • Hadoop’s Azure Blob Storage libraries for deployment with the following versions:
    • 2.9.1+ for Hadoop 2
    • 3.0.1+ for Hadoop 3
  • Apache Spark associated with the corresponding Delta Lake version and compiled with Hadoop version that is compatible with the chosen Hadoop libraries.

For example, a possible combination that will work is Delta 0.7.0 or above, along with Apache Spark 3.0 compiled and deployed with Hadoop 3.2.

Here are the steps to configure Delta Lake on Azure Blob storage.

  1. Include hadoop-azure JAR in the classpath. See the requirements above for version details.

  2. Set up credentials.

    You can set up your credentials in the Spark configuration property.

    We recommend that you use a SAS token. In Scala, you can use the following:

spark.conf.set(
"fs.azure.sas.<your-container-name>.<your-storage-account-name>.blob.core.windows.net",
"<complete-query-string-of-your-sas-for-the-container>")

Or you can specify an account access key:

spark.conf.set(
"fs.azure.account.key.<your-storage-account-name>.blob.core.windows.net",
"<your-storage-account-access-key>")
spark.range(5).write.format("delta").save("wasbs://<your-container-name>@<your-storage-account-name>.blob.core.windows.net/<path-to-delta-table>")
spark.read.format("delta").load("wasbs://<your-container-name>@<your-storage-account-name>.blob.core.windows.net/<path-to-delta-table>").show()
  • A service principal for OAuth 2.0 access
  • Delta Lake 0.2.0 or above
  • Hadoop’s Azure Data Lake Storage Gen1 libraries for deployment with the following versions:
    • 2.9.1+ for Hadoop 2
    • 3.0.1+ for Hadoop 3
  • Apache Spark associated with the corresponding Delta Lake version and compiled with Hadoop version that is compatible with the chosen Hadoop libraries.

Here are the steps to configure Delta Lake on Azure Data Lake Storage Gen1.

  1. Include hadoop-azure-datalake JAR in the classpath. See the requirements above for version details.

  2. Set up Azure Data Lake Storage Gen1 credentials.

    You can set the following Hadoop configurations with your credentials (in Scala):

spark.conf.set("dfs.adls.oauth2.access.token.provider.type", "ClientCredential")
spark.conf.set("dfs.adls.oauth2.client.id", "<your-oauth2-client-id>")
spark.conf.set("dfs.adls.oauth2.credential", "<your-oauth2-credential>")
spark.conf.set("dfs.adls.oauth2.refresh.url", "https://login.microsoftonline.com/<your-directory-id>/oauth2/token")
spark.range(5).write.format("delta").save("adl://<your-adls-account>.azuredatalakestore.net/<path-to-delta-table>")
spark.read.format("delta").load("adl://<your-adls-account>.azuredatalakestore.net/<path-to-delta-table>").show()
  • A service principal for OAuth 2.0 access or a shared key
  • Delta Lake 0.2.0 or above
  • Hadoop’s Azure Data Lake Storage Gen2 libraries for deployment with the following versions:
    • 3.2.0+ for Hadoop 3
  • Apache Spark associated with the corresponding Delta Lake version and compiled with Hadoop version that is compatible with the chosen Hadoop libraries.

Here are the steps to configure Delta Lake on Azure Data Lake Storage Gen2.

  1. Include hadoop-azure and azure-storage JARs in the classpath. See the requirements above for version details.

  2. Set up credentials.

    You can use either OAuth 2.0 with service principal or shared key authentication:

    For OAuth 2.0 with service principal (recommended):

spark.conf.set("fs.azure.account.auth.type.<storage-account>.dfs.core.windows.net", "OAuth")
spark.conf.set("fs.azure.account.oauth.provider.type.<storage-account>.dfs.core.windows.net", "org.apache.hadoop.fs.azurebfs.oauth2.ClientCredsTokenProvider")
spark.conf.set("fs.azure.account.oauth2.client.id.<storage-account>.dfs.core.windows.net", "<application-id>")
spark.conf.set("fs.azure.account.oauth2.client.secret.<storage-account>.dfs.core.windows.net", "<service-credential>")
spark.conf.set("fs.azure.account.oauth2.client.endpoint.<storage-account>.dfs.core.windows.net", "https://login.microsoftonline.com/<directory-id>/oauth2/token")

For shared key authentication:

spark.conf.set("fs.azure.account.key.<storage-account>.dfs.core.windows.net", "<storage-account-access-key>")
spark.range(5).write.format("delta").save("abfss://<container-name>@<storage-account>.dfs.core.windows.net/<path-to-delta-table>")
spark.read.format("delta").load("abfss://<container-name>@<storage-account>.dfs.core.windows.net/<path-to-delta-table>").show()

Delta Lake has built-in support for HDFS with full transactional guarantees for concurrent reads and writes from multiple clusters. No additional configuration is required.

spark.range(5).write.format("delta").save("hdfs://<namenode>:<port>/<path-to-delta-table>")
spark.read.format("delta").load("hdfs://<namenode>:<port>/<path-to-delta-table>").show()

Delta Lake has built-in support for Google Cloud Storage (GCS) with full transactional guarantees for concurrent reads and writes from multiple clusters.

  • Google Cloud Storage credentials
  • Delta Lake 0.2.0 or above
  • Hadoop’s GCS connector for the version of Hadoop that Apache Spark is compiled for
  1. Include the GCS connector JAR in the classpath.

  2. Set up credentials using one of the following methods:

spark.conf.set("google.cloud.auth.service.account.json.keyfile", "<path-to-json-key-file>")
spark.range(5).write.format("delta").save("gs://<bucket>/<path-to-delta-table>")
spark.read.format("delta").load("gs://<bucket>/<path-to-delta-table>").show()

Delta Lake supports Oracle Cloud Infrastructure (OCI) Object Storage with full transactional guarantees for concurrent reads and writes from multiple clusters.

  • OCI credentials
  • Delta Lake 0.2.0 or above
  • Hadoop’s OCI connector for the version of Hadoop that Apache Spark is compiled for
  1. Include the OCI connector JAR in the classpath.

  2. Set up credentials in Spark configuration:

spark.conf.set("fs.oci.client.auth.tenantId", "<tenant-ocid>")
spark.conf.set("fs.oci.client.auth.userId", "<user-ocid>")
spark.conf.set("fs.oci.client.auth.fingerprint", "<api-key-fingerprint>")
spark.conf.set("fs.oci.client.auth.pemfilepath", "<path-to-private-key-file>")
spark.range(5).write.format("delta").save("oci://<bucket>@<namespace>/<path-to-delta-table>")
spark.read.format("delta").load("oci://<bucket>@<namespace>/<path-to-delta-table>").show()

Delta Lake supports IBM Cloud Object Storage with full transactional guarantees for concurrent reads and writes from multiple clusters.

  • IBM Cloud Object Storage credentials
  • Delta Lake 0.2.0 or above
  • Hadoop’s Stocator connector for the version of Hadoop that Apache Spark is compiled for
  1. Include the Stocator connector JAR in the classpath.

  2. Set up credentials in Spark configuration:

spark.conf.set("fs.cos.service.endpoint", "<endpoint-url>")
spark.conf.set("fs.cos.service.access.key", "<access-key>")
spark.conf.set("fs.cos.service.secret.key", "<secret-key>")
spark.range(5).write.format("delta").save("cos://<bucket>.<service>/<path-to-delta-table>")
spark.read.format("delta").load("cos://<bucket>.<service>/<path-to-delta-table>").show()