hide:

  • navigation

Configuration

Setting Configuration Values

There are three ways to pass in configuration:

  • Using the .pyiceberg.yaml configuration file (Recommended)
  • Through environment variables
  • By passing in credentials through the CLI or the Python API

The configuration file can be stored in either the directory specified by the PYICEBERG_HOME environment variable, the home directory, or current working directory (in this order).

To change the path searched for the .pyiceberg.yaml, you can overwrite the PYICEBERG_HOME environment variable.

Another option is through environment variables:

export PYICEBERG_CATALOG__DEFAULT__URI=thrift://localhost:9083
export PYICEBERG_CATALOG__DEFAULT__S3__ACCESS_KEY_ID=username
export PYICEBERG_CATALOG__DEFAULT__S3__SECRET_ACCESS_KEY=password

The environment variable picked up by Iceberg starts with PYICEBERG_ and then follows the yaml structure below, where a double underscore __ represents a nested field, and the underscore _ is converted into a dash -.

For example, PYICEBERG_CATALOG__DEFAULT__S3__ACCESS_KEY_ID, sets s3.access-key-id on the default catalog.

Tables

Iceberg tables support table properties to configure table behavior.

Write options

KeyOptionsDefaultDescription
write.parquet.compression-codec{uncompressed,zstd,gzip,snappy}zstdSets the Parquet compression coddec.
write.parquet.compression-levelIntegernullParquet compression level for the codec. If not set, it is up to PyIceberg
write.parquet.row-group-limitNumber of rows1048576The upper bound of the number of entries within a single row group
write.parquet.page-size-bytesSize in bytes1MBSet a target threshold for the approximate encoded size of data pages within a column chunk
write.parquet.page-row-limitNumber of rows20000Set a target threshold for the maximum number of rows within a column chunk
write.parquet.dict-size-bytesSize in bytes2MBSet the dictionary page size limit per row group
write.metadata.previous-versions-maxInteger100The max number of previous version metadata files to keep before deleting after commit.
write.metadata.delete-after-commit.enabledBooleanFalseWhether to automatically delete old tracked metadata files after each table commit. It will retain a number of the most recent metadata files, which can be set using property write.metadata.previous-versions-max.
write.object-storage.enabledBooleanTrueEnables the ObjectStoreLocationProvider that adds a hash component to file paths. Note: the default value of True differs from Iceberg's Java implementation
write.object-storage.partitioned-pathsBooleanTrueControls whether partition values are included in file paths when object storage is enabled
write.py-location-provider.implString of form module.ClassNamenullOptional, custom LocationProvider implementation
write.data.pathString pointing to location{metadata.location}/dataSets the location under which data is written.
write.metadata.pathString pointing to location{metadata.location}/metadataSets the location under which metadata is written.

Table behavior options

KeyOptionsDefaultDescription
commit.manifest.target-size-bytesSize in bytes8388608 (8MB)Target size when merging manifest files
commit.manifest.min-count-to-mergeNumber of manifests100Target size when merging manifest files
commit.manifest-merge.enabledBooleanFalseControls whether to automatically merge manifests on writes

!!! note “Fast append” Unlike Java implementation, PyIceberg default to the fast append and thus commit.manifest-merge.enabled is set to False by default.

FileIO

Iceberg works with the concept of a FileIO which is a pluggable module for reading, writing, and deleting files. By default, PyIceberg will try to initialize the FileIO that‘s suitable for the scheme (s3://, gs://, etc.) and will use the first one that’s installed.

  • s3, s3a, s3n: PyArrowFileIO, FsspecFileIO
  • gs: PyArrowFileIO
  • file: PyArrowFileIO
  • hdfs: PyArrowFileIO
  • abfs, abfss: FsspecFileIO
  • oss: PyArrowFileIO

You can also set the FileIO explicitly:

KeyExampleDescription
py-io-implpyiceberg.io.fsspec.FsspecFileIOSets the FileIO explicitly to an implementation, and will fail explicitly if it can't be loaded

For the FileIO there are several configuration options available:

S3

KeyExampleDescription
s3.endpointhttps://10.0.19.25/Configure an alternative endpoint of the S3 service for the FileIO to access. This could be used to use S3FileIO with any s3-compatible object storage service that has a different endpoint, or access a private S3 endpoint in a virtual private cloud.
s3.access-key-idadminConfigure the static access key id used to access the FileIO.
s3.secret-access-keypasswordConfigure the static secret access key used to access the FileIO.
s3.session-tokenAQoDYXdzEJr...Configure the static session token used to access the FileIO.
s3.role-session-namesessionAn optional identifier for the assumed role session.
s3.role-arnarn:aws:...AWS Role ARN. If provided instead of access_key and secret_key, temporary credentials will be fetched by assuming this role.
s3.signerbearerConfigure the signature version of the FileIO.
s3.signer.urihttp://my.signer:8080/s3Configure the remote signing uri if it differs from the catalog uri. Remote signing is only implemented for FsspecFileIO. The final request is sent to <s3.signer.uri>/<s3.signer.endpoint>.
s3.signer.endpointv1/main/s3-signConfigure the remote signing endpoint. Remote signing is only implemented for FsspecFileIO. The final request is sent to <s3.signer.uri>/<s3.signer.endpoint>. (default : v1/aws/s3/sign).
s3.regionus-west-2Configure the default region used to initialize an S3FileSystem. PyArrowFileIO attempts to automatically tries to resolve the region if this isn't set (only supported for AWS S3 Buckets).
s3.resolve-regionFalseOnly supported for PyArrowFileIO, when enabled, it will always try to resolve the location of the bucket (only supported for AWS S3 Buckets).
s3.proxy-urihttp://my.proxy.com:8080Configure the proxy server to be used by the FileIO.
s3.connect-timeout60.0Configure socket connection timeout, in seconds.
s3.request-timeout60.0Configure socket read timeouts on Windows and macOS, in seconds.
s3.force-virtual-addressingFalseWhether to use virtual addressing of buckets. If true, then virtual addressing is always enabled. If false, then virtual addressing is only enabled if endpoint_override is empty. This can be used for non-AWS backends that only support virtual hosted-style access.

HDFS

KeyExampleDescription
hdfs.hosthttps://10.0.19.25/Configure the HDFS host to connect to
hdfs.port9000Configure the HDFS port to connect to.
hdfs.useruserConfigure the HDFS username used for connection.
hdfs.kerberos_ticketkerberos_ticketConfigure the path to the Kerberos ticket cache.

Azure Data lake

KeyExampleDescription
adls.connection-stringAccountName=devstoreaccount1;AccountKey=Eby8vdM02xNOcqF...;BlobEndpoint=http://localhost/A connection string. This could be used to use FileIO with any adls-compatible object storage service that has a different endpoint (like azurite).
adls.account-namedevstoreaccount1The account that you want to connect to
adls.account-keyEby8vdM02xNOcqF...The key to authentication against the account.
adls.sas-tokenNuHOuuzdQN7VRM%2FOpOeqBlawRCA845IY05h9eu1Yte4%3DThe shared access signature
adls.tenant-idad667be4-b811-11ed-afa1-0242ac120002The tenant-id
adls.client-idad667be4-b811-11ed-afa1-0242ac120002The client-id
adls.client-secretoCA3R6P*ka#oa1Sms2J74z...The client-secret

Google Cloud Storage

KeyExampleDescription
gcs.project-idmy-gcp-projectConfigure Google Cloud Project for GCS FileIO.
gcs.oauth2.tokenya29.dr.AfM...String representation of the access token used for temporary access.
gcs.oauth2.token-expires-at1690971805918Configure expiration for credential generated with an access token. Milliseconds since epoch
gcs.accessread_onlyConfigure client to have specific access. Must be one of ‘read_only’, ‘read_write’, or ‘full_control’
gcs.consistencymd5Configure the check method when writing files. Must be one of ‘none’, ‘size’, or ‘md5’
gcs.cache-timeout60Configure the cache expiration time in seconds for object metadata cache
gcs.requester-paysFalseConfigure whether to use requester-pays requests
gcs.session-kwargs{}Configure a dict of parameters to pass on to aiohttp.ClientSession; can contain, for example, proxy settings.
gcs.service.hosthttp://0.0.0.0:4443Configure an alternative endpoint for the GCS FileIO to access (format protocol://host:port) If not given, defaults to the value of environment variable “STORAGE_EMULATOR_HOST”; if that is not set either, will use the standard Google endpoint.
gcs.default-locationUSConfigure the default location where buckets are created, like ‘US’ or ‘EUROPE-WEST3’.
gcs.version-awareFalseConfigure whether to support object versioning on the GCS bucket.

Alibaba Cloud Object Storage Service (OSS)

PyIceberg uses S3FileSystem class to connect to OSS bucket as the service is compatible with S3 SDK as long as the endpoint is addressed with virtual hosted style.

KeyExampleDescription
s3.endpointhttps://s3.oss-your-bucket-region.aliyuncs.com/Configure an endpoint of the OSS service for the FileIO to access. Be sure to use S3 compatible endpoint as given in the example.
s3.access-key-idadminConfigure the static access key id used to access the FileIO.
s3.secret-access-keypasswordConfigure the static secret access key used to access the FileIO.
s3.session-tokenAQoDYXdzEJr...Configure the static session token used to access the FileIO.
s3.force-virtual-addressingTrueWhether to use virtual addressing of buckets. This must be set to True as OSS can only be accessed with virtual hosted style address.

PyArrow

KeyExampleDescription
pyarrow.use-large-types-on-readTrueUse large PyArrow types i.e. large_string, large_binary and large_list field types on table scans. The default value is True.

Location Providers

Apache Iceberg uses the concept of a LocationProvider to manage file paths for a table's data files. In PyIceberg, the LocationProvider module is designed to be pluggable, allowing customization for specific use cases, and to additionally determine metadata file locations. The LocationProvider for a table can be specified through table properties.

Both data file and metadata file locations can be customized by configuring the table properties write.data.path and write.metadata.path, respectively.

For more granular control, you can override the LocationProvider's new_data_location and new_metadata_location methods to define custom logic for generating file paths. See Loading a Custom Location Provider.

PyIceberg defaults to the ObjectStoreLocationProvider, which generates file paths for data files that are optimized for object storage.

Simple Location Provider

The SimpleLocationProvider provides paths prefixed by {location}/data/, where location comes from the table metadata. This can be overridden by setting write.data.path table configuration.

For example, a non-partitioned table might have a data file with location:

s3://bucket/ns/table/data/0000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet

When the table is partitioned, files under a given partition are grouped into a subdirectory, with that partition key and value as the directory name - this is known as the Hive-style partition path format. For example, a table partitioned over a string column category might have a data file with location:

s3://bucket/ns/table/data/category=orders/0000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet

The SimpleLocationProvider is enabled for a table by explicitly setting its write.object-storage.enabled table property to False.

Object Store Location Provider

PyIceberg offers the ObjectStoreLocationProvider, and an optional partition-exclusion optimization, designed for tables stored in object storage. For additional context and motivation concerning these configurations, see their documentation for Iceberg's Java implementation.

When several files are stored under the same prefix, cloud object stores such as S3 often throttle requests on prefixes, resulting in slowdowns. The ObjectStoreLocationProvider counteracts this by injecting deterministic hashes, in the form of binary directories, into file paths, to distribute files across a larger number of object store prefixes.

Paths are prefixed by {location}/data/, where location comes from the table metadata, in a similar manner to the SimpleLocationProvider. This can be overridden by setting write.data.path table configuration.

For example, a table partitioned over a string column category might have a data file with location: (note the additional binary directories)

s3://bucket/ns/table/data/0101/0110/1001/10110010/category=orders/0000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet

The write.object-storage.enabled table property determines whether the ObjectStoreLocationProvider is enabled for a table. It is used by default.

Partition Exclusion

When the ObjectStoreLocationProvider is used, the table property write.object-storage.partitioned-paths, which defaults to True, can be set to False as an additional optimization for object stores. This omits partition keys and values from data file paths entirely to further reduce key size. With it disabled, the same data file above would instead be written to: (note the absence of category=orders)

s3://bucket/ns/table/data/1101/0100/1011/00111010-00000-0-5affc076-96a4-48f2-9cd2-d5efbc9f0c94-00001.parquet

Loading a Custom Location Provider

Similar to FileIO, a custom LocationProvider may be provided for a table by concretely subclassing the abstract base class LocationProvider.

The table property write.py-location-provider.impl should be set to the fully-qualified name of the custom LocationProvider (i.e. mymodule.MyLocationProvider). Recall that a LocationProvider is configured per-table, permitting different location provision for different tables. Note also that Iceberg's Java implementation uses a different table property, write.location-provider.impl, for custom Java implementations.

An example, custom LocationProvider implementation is shown below.

import uuid

class UUIDLocationProvider(LocationProvider):
    def __init__(self, table_location: str, table_properties: Properties):
        super().__init__(table_location, table_properties)

    def new_data_location(self, data_file_name: str, partition_key: Optional[PartitionKey] = None) -> str:
        # Can use any custom method to generate a file path given the partitioning information and file name
        prefix = f"{self.table_location}/{uuid.uuid4()}"
        return f"{prefix}/{partition_key.to_path()}/{data_file_name}" if partition_key else f"{prefix}/{data_file_name}"

Catalogs

PyIceberg currently has native catalog type support for REST, SQL, Hive, Glue and DynamoDB. Alternatively, you can also directly set the catalog implementation:

KeyExampleDescription
typerestType of catalog, one of rest, sql, hive, glue, dymamodb. Default to rest
py-catalog-implmypackage.mymodule.MyCatalogSets the catalog explicitly to an implementation, and will fail explicitly if it can't be loaded

REST Catalog

catalog:
  default:
    uri: http://rest-catalog/ws/
    credential: t-1234:secret

  default-mtls-secured-catalog:
    uri: https://rest-catalog/ws/
    ssl:
      client:
        cert: /absolute/path/to/client.crt
        key: /absolute/path/to/client.key
      cabundle: /absolute/path/to/cabundle.pem
KeyExampleDescription
urihttps://rest-catalog/wsURI identifying the REST Server
ugit-1234:secretHadoop UGI for Hive client.
credentialt-1234:secretCredential to use for OAuth2 credential flow when initializing the catalog
tokenFEW23.DFSDF.FSDFBearer token value to use for Authorization header
scopeopenid offline corpds:ds:profileDesired scope of the requested security token (default : catalog)
resourcerest_catalog.iceberg.comURI for the target resource or service
audiencerest_catalogLogical name of target resource or service
rest.sigv4-enabledtrueSign requests to the REST Server using AWS SigV4 protocol
rest.signing-regionus-east-1The region to use when SigV4 signing a request
rest.signing-nameexecute-apiThe service signing name to use when SigV4 signing a request
oauth2-server-urihttps://auth-service/ccAuthentication URL to use for client credentials authentication (default: uri + ‘v1/oauth/tokens’)

Headers in RESTCatalog

To configure custom headers in RESTCatalog, include them in the catalog properties with the prefix header.. This ensures that all HTTP requests to the REST service include the specified headers.

catalog:
  default:
    uri: http://rest-catalog/ws/
    credential: t-1234:secret
    header.content-type: application/vnd.api+json

Specific headers defined by the RESTCatalog spec include:

KeyOptionsDefaultDescription
header.X-Iceberg-Access-Delegation{vended-credentials,remote-signing}vended-credentialsSignal to the server that the client supports delegated access via a comma-separated list of access mechanisms. The server may choose to supply access via any or none of the requested mechanisms

SQL Catalog

The SQL catalog requires a database for its backend. PyIceberg supports PostgreSQL and SQLite through psycopg2. The database connection has to be configured using the uri property. The init_catalog_tables is optional and defaults to True. If it is set to False, the catalog tables will not be created when the SQLCatalog is initialized. See SQLAlchemy's documentation for URL format:

For PostgreSQL:

catalog:
  default:
    type: sql
    uri: postgresql+psycopg2://username:password@localhost/mydatabase
    init_catalog_tables: false

In the case of SQLite:

!!! warning inline end “Development only” SQLite is not built for concurrency, you should use this catalog for exploratory or development purposes.

catalog:
  default:
    type: sql
    uri: sqlite:////tmp/pyiceberg.db
    init_catalog_tables: false
KeyExampleDefaultDescription
uripostgresql+psycopg2://username:password@localhost/mydatabaseSQLAlchemy backend URL for the catalog database (see documentation for URL format)
echotruefalseSQLAlchemy engine echo param to log all statements to the default log handler
pool_pre_pingtruefalseSQLAlchemy engine pool_pre_ping param to test connections for liveness upon each checkout

In Memory Catalog

The in-memory catalog is built on top of SqlCatalog and uses SQLite in-memory database for its backend.

It is useful for test, demo, and playground but not in production as it does not support concurrent access.

catalog:
  default:
    type: in-memory
    warehouse: /tmp/pyiceberg/warehouse
KeyExampleDefaultDescription
warehouse/tmp/pyiceberg/warehousefile:///tmp/iceberg/warehouseThe directory where the in-memory catalog will store its data files.

Hive Catalog

catalog:
  default:
    uri: thrift://localhost:9083
    s3.endpoint: http://localhost:9000
    s3.access-key-id: admin
    s3.secret-access-key: password
KeyExampleDescription
hive.hive2-compatibletrueUsing Hive 2.x compatibility mode
hive.kerberos-authenticationtrueUsing authentication via Kerberos

When using Hive 2.x, make sure to set the compatibility flag:

catalog:
  default:
...
    hive.hive2-compatible: true

Glue Catalog

Your AWS credentials can be passed directly through the Python API. Otherwise, please refer to How to configure AWS credentials to set your AWS account credentials locally.

catalog:
  default:
    type: glue
    glue.access-key-id: <ACCESS_KEY_ID>
    glue.secret-access-key: <SECRET_ACCESS_KEY>
    glue.session-token: <SESSION_TOKEN>
    glue.region: <REGION_NAME>
    s3.endpoint: http://localhost:9000
    s3.access-key-id: admin
    s3.secret-access-key: password
catalog:
  default:
    type: glue
    glue.profile-name: <PROFILE_NAME>
    glue.region: <REGION_NAME>
    s3.endpoint: http://localhost:9000
    s3.access-key-id: admin
    s3.secret-access-key: password

!!! Note “Client-specific Properties” glue.* properties are for Glue Catalog only. If you want to use the same credentials for both Glue Catalog and S3 FileIO, you can set the client.* properties. See the Unified AWS Credentials section for more details.

KeyExampleDescription
glue.id111111111111Configure the 12-digit ID of the Glue Catalog
glue.skip-archivetrueConfigure whether to skip the archival of older table versions. Default to true
glue.endpointhttps://glue.us-east-1.amazonaws.comConfigure an alternative endpoint of the Glue service for GlueCatalog to access
glue.profile-namedefaultConfigure the static profile used to access the Glue Catalog
glue.regionus-east-1Set the region of the Glue Catalog
glue.access-key-idadminConfigure the static access key id used to access the Glue Catalog
glue.secret-access-keypasswordConfigure the static secret access key used to access the Glue Catalog
glue.session-tokenAQoDYXdzEJr...Configure the static session token used to access the Glue Catalog
glue.max-retries10Configure the maximum number of retries for the Glue service calls
glue.retry-modestandardConfigure the retry mode for the Glue service. Default to standard.

!!! warning “Removed Properties” The properties profile_name, region_name, aws_access_key_id, aws_secret_access_key, and aws_session_token were deprecated and removed in 0.8.0

DynamoDB Catalog

If you want to use AWS DynamoDB as the catalog, you can use the last two ways to configure the pyiceberg and refer How to configure AWS credentials to set your AWS account credentials locally. If you want to use the same credentials for both Dynamodb Catalog and S3 FileIO, you can set the client.* properties.

catalog:
  default:
    type: dynamodb
    table-name: iceberg

If you prefer to pass the credentials explicitly to the client instead of relying on environment variables,

catalog:
  default:
    type: dynamodb
    table-name: iceberg
    dynamodb.access-key-id: <ACCESS_KEY_ID>
    dynamodb.secret-access-key: <SECRET_ACCESS_KEY>
    dynamodb.session-token: <SESSION_TOKEN>
    dynamodb.region: <REGION_NAME>
    s3.endpoint: http://localhost:9000
    s3.access-key-id: admin
    s3.secret-access-key: password

!!! Note “Client-specific Properties” dynamodb.* properties are for DynamoDB Catalog only. If you want to use the same credentials for both DynamoDB Catalog and S3 FileIO, you can set the client.* properties. See the Unified AWS Credentials section for more details.

KeyExampleDescription
dynamodb.profile-namedefaultConfigure the static profile used to access the DynamoDB Catalog
dynamodb.regionus-east-1Set the region of the DynamoDB Catalog
dynamodb.access-key-idadminConfigure the static access key id used to access the DynamoDB Catalog
dynamodb.secret-access-keypasswordConfigure the static secret access key used to access the DynamoDB Catalog
dynamodb.session-tokenAQoDYXdzEJr...Configure the static session token used to access the DynamoDB Catalog

!!! warning “Removed Properties” The properties profile_name, region_name, aws_access_key_id, aws_secret_access_key, and aws_session_token were deprecated and removed in 0.8.0

Custom Catalog Implementations

If you want to load any custom catalog implementation, you can set catalog configurations like the following:

catalog:
  default:
    py-catalog-impl: mypackage.mymodule.MyCatalog
    custom-key1: value1
    custom-key2: value2

Unified AWS Credentials

You can explicitly set the AWS credentials for both Glue/DynamoDB Catalog and S3 FileIO by configuring client.* properties. For example:

catalog:
  default:
    type: glue
    client.access-key-id: <ACCESS_KEY_ID>
    client.secret-access-key: <SECRET_ACCESS_KEY>
    client.region: <REGION_NAME>

configures the AWS credentials for both Glue Catalog and S3 FileIO.

KeyExampleDescription
client.regionus-east-1Set the region of both the Glue/DynamoDB Catalog and the S3 FileIO
client.access-key-idadminConfigure the static access key id used to access both the Glue/DynamoDB Catalog and the S3 FileIO
client.secret-access-keypasswordConfigure the static secret access key used to access both the Glue/DynamoDB Catalog and the S3 FileIO
client.session-tokenAQoDYXdzEJr...Configure the static session token used to access both the Glue/DynamoDB Catalog and the S3 FileIO
client.role-session-namesessionAn optional identifier for the assumed role session.
client.role-arnarn:aws:...AWS Role ARN. If provided instead of access_key and secret_key, temporary credentials will be fetched by assuming this role.

!!! Note “Properties Priority” client.* properties will be overridden by service-specific properties if they are set. For example, if client.region is set to us-west-1 and s3.region is set to us-east-1, the S3 FileIO will use us-east-1 as the region.

Concurrency

PyIceberg uses multiple threads to parallelize operations. The number of workers can be configured by supplying a max-workers entry in the configuration file, or by setting the PYICEBERG_MAX_WORKERS environment variable. The default value depends on the system hardware and Python version. See the Python documentation for more details.

Backward Compatibility

Previous versions of Java (<1.4.0) implementations incorrectly assume the optional attribute current-snapshot-id to be a required attribute in TableMetadata. This means that if current-snapshot-id is missing in the metadata file (e.g. on table creation), the application will throw an exception without being able to load the table. This assumption has been corrected in more recent Iceberg versions. However, it is possible to force PyIceberg to create a table with a metadata file that will be compatible with previous versions. This can be configured by setting the legacy-current-snapshot-id property as “True” in the configuration file, or by setting the PYICEBERG_LEGACY_CURRENT_SNAPSHOT_ID environment variable. Refer to the PR discussion for more details on the issue

Nanoseconds Support

PyIceberg currently only supports upto microsecond precision in its TimestampType. PyArrow timestamp types in ‘s’ and ‘ms’ will be upcast automatically to ‘us’ precision timestamps on write. Timestamps in ‘ns’ precision can also be downcast automatically on write if desired. This can be configured by setting the downcast-ns-timestamp-to-us-on-write property as “True” in the configuration file, or by setting the PYICEBERG_DOWNCAST_NS_TIMESTAMP_TO_US_ON_WRITE environment variable. Refer to the nanoseconds timestamp proposal document for more details on the long term roadmap for nanoseconds support