hide:

  • navigation

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.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 approximate encoded size of data pages within a column chunk
write.parquet.dict-size-bytesSize in bytes2MBSet the dictionary page size limit per row group
write.parquet.row-group-limitNumber of rows122880The Parquet row group limit

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

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.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.singer.uri>/v1/aws/s3/sign.
s3.regionus-west-2Sets the region of the bucket
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.

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

Catalogs

PyIceberg currently has native support for REST, SQL, Hive, Glue and DynamoDB.

There are three ways to pass in configuration:

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

The configuration file is recommended since that's the easiest way to manage the credentials.

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.

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
rest.authorization-urlhttps://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

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. See SQLAlchemy's documentation for URL format:

For PostgreSQL:

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

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

Hive Catalog

catalog:
  default:
    uri: thrift://localhost:9083
    s3.endpoint: http://localhost:9000
    s3.access-key-id: admin
    s3.secret-access-key: password

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

!!! warning “Deprecated Properties” profile_name, region_name, botocore_session, aws_access_key_id, aws_secret_access_key, aws_session_token are deprecated and will be 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 “Deprecated Properties” profile_name, region_name, botocore_session, aws_access_key_id, aws_secret_access_key, aws_session_token are deprecated and will be removed in 0.8.0:

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

!!! 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