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.

    Note

    • Delta Lake on local file system may not support concurrent transactional writes. This is because the local file system may or may not provide atomic renames. So you should not use the local file system for testing concurrent writes.

    • Before version 1.0, Delta Lake supported configuring LogStores by setting spark.delta.logStore.class. This approach is now deprecated. Setting this configuration will use the configured LogStore for all paths, thereby disabling the dynamic scheme-based delegation.

In this article:

Amazon S3

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

Single-cluster

Multi-cluster

Configuration

Comes with Delta Lake out-of-the-box

Is experimental and requires extra configuration

Reads

Supports concurrent reads from multiple clusters

Supports concurrent reads from multiple clusters

Writes

Supports concurrent writes from a single Spark driver

Supports multi-cluster writes

Permissions

S3 credentials

S3 and DynamoDB operating permissions

Single-cluster setup (default)

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.

Warning

Concurrent writes to the same Delta table from multiple Spark drivers can lead to data loss.

Requirements (S3 single-cluster)

  • 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.

Quickstart (S3 single-cluster)

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.2.1 which is pre-built for Hadoop 3.3.1):

    bin/spark-shell \
     --packages io.delta:delta-core_2.12:2.0.0,org.apache.hadoop:hadoop-aws:3.3.1 \
     --conf spark.hadoop.fs.s3a.access.key=<your-s3-access-key> \
     --conf spark.hadoop.fs.s3a.secret.key=<your-s3-secret-key>
    
  2. 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.

Configuration (S3 single-cluster)

Here are the steps to configure Delta Lake for S3.

  1. Include hadoop-aws JAR in the classpath.

    Delta Lake needs the org.apache.hadoop.fs.s3a.S3AFileSystem class from the hadoop-aws package, which implements Hadoop’s FileSystem API for S3. Make sure the version of this package matches the Hadoop version with which Spark was built.

  2. Set up S3 credentials.

    We recommend using IAM roles for authentication and authorization. But if you want to use keys, here is one way is to set up the Hadoop configurations (in Scala):

    sc.hadoopConfiguration.set("fs.s3a.access.key", "<your-s3-access-key>")
    sc.hadoopConfiguration.set("fs.s3a.secret.key", "<your-s3-secret-key>")
    

Multi-cluster setup

Note

This support is new and experimental.

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.

Warning

This multi-cluster writing solution is only safe when all writers use this LogStore implementation as well as the same DynamoDB table and region. If some drivers use out-of-the-box Delta Lake while others use this experimental LogStore, then data loss can occur.

Requirements (S3 multi-cluster)

Quickstart (S3 multi-cluster)

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.2.1 which is pre-built for Hadoop 3.3.1):

    bin/spark-shell \
     --packages io.delta:delta-core_2.12:2.0.0,org.apache.hadoop:hadoop-aws:3.3.1,io.delta:delta-storage-s3-dynamodb:2.0.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
    
  2. 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()
    

Setup Configuration (S3 multi-cluster)

  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.

      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
      

      Note: once you select a table-name and region, you will have to specify them in each Spark session in order for this multi-cluster mode to work correctly. See table below.

    • Automatic DynamoDB table creation

      Nonetheless, after specifying this LogStore implementation, if the default DynamoDB table does not already exist, then it will be created for you automatically. This default table supports 5 strongly consistent reads and 5 writes per second. You may change these default values using the table-creation-only configurations keys detailed in the table below.

  2. Follow the configuration steps listed in Configuration (S3 single-cluster) section.

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

  4. 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:

    Configuration Key

    Description

    Default

    spark.io.delta.storage.S3DynamoDBLogStore.ddb.tableName

    The name of the DynamoDB table to use

    delta_log

    spark.io.delta.storage.S3DynamoDBLogStore.ddb.region

    The region to be used by the client

    us-east-1

    spark.io.delta.storage.S3DynamoDBLogStore.credentials.provider

    The AWSCredentialsProvider* used by the client

    DefaultAWSCredentialsProviderChain

    spark.io.delta.storage.S3DynamoDBLogStore.provisionedThroughput.rcu

    (Table-creation-only**) Read Capacity Units

    5

    spark.io.delta.storage.S3DynamoDBLogStore.provisionedThroughput.wcu

    (Table-creation-only**) Write Capacity Units

    5

    • *For more details on AWS credential providers, see the AWS documentation.

    • **These configurations are only used when the given DynamoDB table doesn’t already exist and needs to be automatically created.

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:

    aws dynamodb update-time-to-live \
      --region us-east-1 \
      --table-name delta_log \
      --time-to-live-specification "Enabled=true, AttributeName=commitTime"
    
  3. 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>:

      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
      
    2. 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.

      Note: It is important that you choose a sufficiently large expiration value. As stated above, the latest temp file will be used during recovery of a failed commit. If this temp file is deleted, then your DynamoDB table and S3 <delta_table_path>/_delta_log/.tmp/ folder will be out of sync.

      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
            }
          }
        ]
      }
      
      aws s3api put-bucket-lifecycle-configuration \
        --bucket my-bucket \
        --lifecycle-configuration file://lifecycle.json
      

Note

AWS S3 may have a limit on the number of rules per bucket. See PutBucketLifecycleConfiguration for details.

Microsoft Azure storage

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:

Azure Blob storage

Requirements (Azure Blob storage)

  • 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 (see the Quick Start page of the relevant Delta version’s documentation) 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.

Configuration (Azure Blob storage)

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>")
    

Usage (Azure Blob storage)

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()

Azure Data Lake Storage Gen1

Requirements (ADLS Gen1)

  • 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 (see the Quick Start page of the relevant Delta version’s documentation) 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.

Configuration (ADLS Gen1)

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")
    

Usage (ADLS Gen1)

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()

Azure Data Lake Storage Gen2

Requirements (ADLS Gen2)

  • Account created in Azure Data Lake Storage Gen2)

  • Service principal created and assigned the Storage Blob Data Contributor role for the storage account.

    • Note the storage-account-name, directory-id (also known as tenant-id), application-id, and password of the principal. These will be used for configuring Spark.

  • Delta Lake 0.7.0 or above

  • Apache Spark 3.0 or above

  • Apache Spark used must be built with Hadoop 3.2 or above.

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.

Configuration (ADLS Gen2)

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

  1. Include the JAR of the Maven artifact hadoop-azure-datalake in the classpath. See the requirements for version details. In addition, you may also have to include JARs for Maven artifacts hadoop-azure and wildfly-openssl.

  2. Set up Azure Data Lake Storage Gen2 credentials.

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

    where <storage-account-name>, <application-id>, <directory-id> and <password> are details of the service principal we set as requirements earlier.

  3. Initialize the file system if needed

spark.conf.set("fs.azure.createRemoteFileSystemDuringInitialization", "true")
dbutils.fs.ls("abfss://<container-name>@<storage-account-name>.dfs.core.windows.net/")
spark.conf.set("fs.azure.createRemoteFileSystemDuringInitialization", "false")

Usage (ADLS Gen2)

spark.range(5).write.format("delta").save("abfss://<container-name>@<storage-account-name>.dfs.core.windows.net/<path-to-delta-table>")

spark.read.format("delta").load("abfss://<container-name>@<storage-account-name>.dfs.core.windows.net/<path-to-delta-table>").show()

where <container-name> is the file system name under the container.

HDFS

Delta Lake has built-in support for HDFS with full transactional guarantees on concurrent reads and writes from multiple clusters. See Hadoop and Spark documentation for configuring credentials.

Google Cloud Storage

You must configure Delta Lake to use the correct LogStore for concurrently reading and writing from GCS.

Requirements (GCS)

Configuration (GCS)

  1. For Delta Lake 1.2.0 and below, you must explicitly configure the LogStore implementation for the scheme gs.

    spark.delta.logStore.gs.impl=io.delta.storage.GCSLogStore
    
  2. Include the JAR for gcs-connector in the classpath. See the documentation for details on how to configure Spark with GCS.

Usage (GCS)

spark.range(5, 10).write.format("delta").mode("append").save("gs://<bucket-name>/<path-to-delta-table>")

spark.read.format("delta").load("gs://<bucket-name>/<path-to-delta-table>").show()

Oracle Cloud Infrastructure

Note

This support is new and experimental.

You have to configure Delta Lake to use the correct LogStore for concurrently reading and writing.

Configuration (OCI)

  1. Configure LogStore implementation for the scheme oci.

    spark.delta.logStore.oci.impl=io.delta.storage.OracleCloudLogStore
    
  2. Include the JARs for delta-contribs and hadoop-oci-connector in the classpath. See Using the HDFS Connector with Spark for details on how to configure Spark with OCI.

  3. Set the OCI Object Store credentials as explained in the documentation.

Usage (OCI)

spark.range(5).write.format("delta").save("oci://<ociBucket>@<ociNameSpace>/<path-to-delta-table>")

spark.read.format("delta").load("oci://<ociBucket>@<ociNameSpace>/<path-to-delta-table>").show()

IBM Cloud Object Storage

Note

This support is new and experimental.

You have to configure Delta Lake to use the correct LogStore for concurrently reading and writing.

Requirements (IBM)

Configuration (IBM)

  1. Configure LogStore implementation for the scheme cos.

    spark.delta.logStore.cos.impl=io.delta.storage.IBMCOSLogStore
    
  2. Include the JARs for delta-contribs and Stocator in the classpath.

  3. Configure Stocator with atomic write support by setting the following properties in the Hadoop configuration.

    fs.stocator.scheme.list=cos
    fs.cos.impl=com.ibm.stocator.fs.ObjectStoreFileSystem
    fs.stocator.cos.impl=com.ibm.stocator.fs.cos.COSAPIClient
    fs.stocator.cos.scheme=cos
    fs.cos.atomic.write=true
    
  4. Set up IBM COS credentials. The example below uses access keys with a service named service (in Scala):

    sc.hadoopConfiguration.set("fs.cos.service.endpoint", "<your-cos-endpoint>")
    sc.hadoopConfiguration.set("fs.cos.service.access.key", "<your-cos-access-key>")
    sc.hadoopConfiguration.set("fs.cos.service.secret.key", "<your-cos-secret-key>")
    

Usage (IBM)

spark.range(5).write.format("delta").save("cos://<your-cos-bucket>.service/<path-to-delta-table>")
spark.read.format("delta").load("cos://<your-cos-bucket>.service/<path-to-delta-table>").show()