blob: 89e755dc0dd1ece11aef64261b8145e6bb997935 [file] [log] [blame]
<!doctype html><html><head><meta charset=utf-8><meta http-equiv=X-UA-Compatible content="IE=edge"><meta name=viewport content="width=device-width,initial-scale=1"><meta name=description content><meta name=author content><title>Configuration</title>
<link href=../css/bootstrap.css rel=stylesheet><link href=../css/markdown.css rel=stylesheet><link href=../css/katex.min.css rel=stylesheet><link href=../css/iceberg-theme.css rel=stylesheet><link href=../font-awesome-4.7.0/css/font-awesome.min.css rel=stylesheet type=text/css><link href="//fonts.googleapis.com/css?family=Lato:300,400,700,300italic,400italic,700italic" rel=stylesheet type=text/css><link href=../css/termynal.css rel=stylesheet></head><body><head><script>function addAnchor(e){e.insertAdjacentHTML("beforeend",`<a href="#${e.id}" class="anchortag" ariaLabel="Anchor"> 🔗 </a>`)}document.addEventListener("DOMContentLoaded",function(){var e=document.querySelectorAll("h1[id], h2[id], h3[id], h4[id]");e&&e.forEach(addAnchor)})</script></head><nav class="navbar navbar-default" role=navigation><topsection><div class=navbar-fixed-top><div><button type=button class=navbar-toggle data-toggle=collapse data-target=div.sidebar>
<span class=sr-only>Toggle navigation</span>
<span class=icon-bar></span>
<span class=icon-bar></span>
<span class=icon-bar></span>
</button>
<a class="page-scroll navbar-brand" href=https://iceberg.apache.org/><img class=top-navbar-logo src=https://iceberg.apache.org/docs/fd-update-slack-url//img/iceberg-logo-icon.png> Apache Iceberg</a></div><div><input type=search class=form-control id=search-input placeholder=Search... maxlength=64 data-hotkeys=s/></div><div class=versions-dropdown><span>1.4.2</span> <i class="fa fa-chevron-down"></i><div class=versions-dropdown-content><ul><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../latest>latest</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.4.2>1.4.2</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.4.1>1.4.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.4.0>1.4.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.3.1>1.3.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.3.0>1.3.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.2.1>1.2.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.2.0>1.2.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.1.0>1.1.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../1.0.0>1.0.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../0.14.1>0.14.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../0.14.0>0.14.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../0.13.2>0.13.2</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../0.13.1>0.13.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../0.13.0>0.13.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../0.12.1>0.12.1</a></li></ul></div></div></div><div class="navbar-menu-fixed-top navbar-pages-group"><div class=versions-dropdown><div class=topnav-page-selection><a href>Quickstart</a> <i class="fa fa-chevron-down"></i></div class="topnav-page-selection"><div class=versions-dropdown-content><ul><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../hive-quickstart>Hive</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../spark-quickstart>Spark</a></li class="topnav-page-selection"></ul></div></div><div class=topnav-page-selection><a id=active href=https://iceberg.apache.org/docs/fd-update-slack-url/../../docs/latest>Docs</a></div><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../releases>Releases</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../roadmap>Roadmap</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../blogs>Blogs</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../talks>Talks</a></div class="topnav-page-selection"><div class=versions-dropdown><div class=topnav-page-selection><a href>Project</a> <i class="fa fa-chevron-down"></i></div class="topnav-page-selection"><div class=versions-dropdown-content><ul><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../community>Community</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../spec>Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../view-spec>View Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../puffin-spec>Puffin Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../multi-engine-support>Multi-Engine Support</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../how-to-release>How To Release</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../terms>Terms</a></li class="topnav-page-selection"></ul></div></div><div class=versions-dropdown><div class=topnav-page-selection><a href>Concepts</a> <i class="fa fa-chevron-down"></i></div class="topnav-page-selection"><div class=versions-dropdown-content><ul><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../catalog>Catalogs</a></li class="topnav-page-selection"></ul></div></div><div class=versions-dropdown><div class=topnav-page-selection><a href>ASF</a> <i class="fa fa-chevron-down"></i></div class="topnav-page-selection"><div class=versions-dropdown-content><ul><li class=topnav-page-selection><a target=_blank href=https://www.apache.org/foundation/sponsorship.html>Donate</a></li class="topnav-page-selection"><li class=topnav-page-selection><a target=_blank href=https://www.apache.org/events/current-event.html>Events</a></li class="topnav-page-selection"><li class=topnav-page-selection><a target=_blank href=https://www.apache.org/licenses/>License</a></li class="topnav-page-selection"><li class=topnav-page-selection><a target=_blank href=https://www.apache.org/security/>Security</a></li class="topnav-page-selection"><li class=topnav-page-selection><a target=_blank href=https://www.apache.org/foundation/thanks.html>Sponsors</a></li class="topnav-page-selection"></ul></div></div><div class=topnav-page-selection><a href=https://github.com/apache/iceberg target=_blank><img src=https://iceberg.apache.org/docs/fd-update-slack-url//img/GitHub-Mark.png target=_blank class=top-navbar-logo></a></div><div class=topnav-page-selection><a href=https://join.slack.com/t/apache-iceberg/shared_invite/zt-27f22riz7-o8nCsl5Vbc_2h6~3DF6qlw target=_blank><img src=https://iceberg.apache.org/docs/fd-update-slack-url//img/Slack_Mark_Web.png target=_blank class=top-navbar-logo></a></div></div></topsection></nav><section><div id=search-results-container><ul id=search-results></ul></div></section><body dir=ltr><section><div class="grid-container leftnav-and-toc"><div class="sidebar markdown-body"><div id=full><ul><li><a href=../><span>Introduction</span></a></li><li><a class=chevron-toggle data-toggle=collapse data-parent=full href=#Tables><span>Tables</span>
<i class="fa fa-chevron-right"></i>
<i class="fa fa-chevron-down"></i></a></li><div id=Tables class="collapse in"><ul class=sub-menu><li><a href=../branching/>Branching and Tagging</a></li><li><a id=active href=../configuration/>Configuration</a></li><li><a href=../evolution/>Evolution</a></li><li><a href=../maintenance/>Maintenance</a></li><li><a href=../metrics-reporting/>Metrics Reporting</a></li><li><a href=../partitioning/>Partitioning</a></li><li><a href=../performance/>Performance</a></li><li><a href=../reliability/>Reliability</a></li><li><a href=../schemas/>Schemas</a></li></ul></div><li><a class="chevron-toggle collapsed" data-toggle=collapse data-parent=full href=#Spark><span>Spark</span>
<i class="fa fa-chevron-right"></i>
<i class="fa fa-chevron-down"></i></a></li><div id=Spark class=collapse><ul class=sub-menu><li><a href=../getting-started/>Getting Started</a></li><li><a href=../spark-configuration/>Configuration</a></li><li><a href=../spark-ddl/>DDL</a></li><li><a href=../spark-procedures/>Procedures</a></li><li><a href=../spark-queries/>Queries</a></li><li><a href=../spark-structured-streaming/>Structured Streaming</a></li><li><a href=../spark-writes/>Writes</a></li></ul></div><li><a class="chevron-toggle collapsed" data-toggle=collapse data-parent=full href=#Flink><span>Flink</span>
<i class="fa fa-chevron-right"></i>
<i class="fa fa-chevron-down"></i></a></li><div id=Flink class=collapse><ul class=sub-menu><li><a href=../flink/>Flink Getting Started</a></li><li><a href=../flink-connector/>Flink Connector</a></li><li><a href=../flink-ddl/>Flink DDL</a></li><li><a href=../flink-queries/>Flink Queries</a></li><li><a href=../flink-writes/>Flink Writes</a></li><li><a href=../flink-actions/>Flink Actions</a></li><li><a href=../flink-configuration/>Flink Configuration</a></li></ul></div><li><a href=../hive/><span>Hive</span></a></li><li><a target=_blank href=https://trino.io/docs/current/connector/iceberg.html><span>Trino</span></a></li><li><a target=_blank href=https://clickhouse.com/docs/en/engines/table-engines/integrations/iceberg><span>ClickHouse</span></a></li><li><a target=_blank href=https://prestodb.io/docs/current/connector/iceberg.html><span>Presto</span></a></li><li><a target=_blank href=https://docs.dremio.com/data-formats/apache-iceberg/><span>Dremio</span></a></li><li><a target=_blank href=https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog><span>StarRocks</span></a></li><li><a target=_blank href=https://docs.aws.amazon.com/athena/latest/ug/querying-iceberg.html><span>Amazon Athena</span></a></li><li><a target=_blank href=https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-iceberg-use-cluster.html><span>Amazon EMR</span></a></li><li><a target=_blank href=https://impala.apache.org/docs/build/html/topics/impala_iceberg.html><span>Impala</span></a></li><li><a target=_blank href=https://doris.apache.org/docs/dev/lakehouse/multi-catalog/iceberg><span>Doris</span></a></li><li><a class="chevron-toggle collapsed" data-toggle=collapse data-parent=full href=#Integrations><span>Integrations</span>
<i class="fa fa-chevron-right"></i>
<i class="fa fa-chevron-down"></i></a></li><div id=Integrations class=collapse><ul class=sub-menu><li><a href=../aws/>AWS</a></li><li><a href=../dell/>Dell</a></li><li><a href=../jdbc/>JDBC</a></li><li><a href=../nessie/>Nessie</a></li></ul></div><li><a class="chevron-toggle collapsed" data-toggle=collapse data-parent=full href=#API><span>API</span>
<i class="fa fa-chevron-right"></i>
<i class="fa fa-chevron-down"></i></a></li><div id=API class=collapse><ul class=sub-menu><li><a href=../java-api-quickstart/>Java Quickstart</a></li><li><a href=../api/>Java API</a></li><li><a href=../custom-catalog/>Java Custom Catalog</a></li></ul></div><li><a class="chevron-toggle collapsed" data-toggle=collapse data-parent=full href=#Migration><span>Migration</span>
<i class="fa fa-chevron-right"></i>
<i class="fa fa-chevron-down"></i></a></li><div id=Migration class=collapse><ul class=sub-menu><li><a href=../table-migration/>Overview</a></li><li><a href=../hive-migration/>Hive Migration</a></li><li><a href=../delta-lake-migration/>Delta Lake Migration</a></li></ul></div><li><a href=https://iceberg.apache.org/docs/fd-update-slack-url/../../javadoc/latest><span>Javadoc</span></a></li><li><a target=_blank href=https://py.iceberg.apache.org/><span>PyIceberg</span></a></li></div></div><div id=content class=markdown-body><div class=margin-for-toc><h1 id=configuration>Configuration</h1><h2 id=table-properties>Table properties</h2><p>Iceberg tables support table properties to configure table behavior, like the default split size for readers.</p><h3 id=read-properties>Read properties</h3><table><thead><tr><th>Property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>read.split.target-size</td><td>134217728 (128 MB)</td><td>Target size when combining data input splits</td></tr><tr><td>read.split.metadata-target-size</td><td>33554432 (32 MB)</td><td>Target size when combining metadata input splits</td></tr><tr><td>read.split.planning-lookback</td><td>10</td><td>Number of bins to consider when combining input splits</td></tr><tr><td>read.split.open-file-cost</td><td>4194304 (4 MB)</td><td>The estimated cost to open a file, used as a minimum weight when combining splits.</td></tr><tr><td>read.parquet.vectorization.enabled</td><td>true</td><td>Controls whether Parquet vectorized reads are used</td></tr><tr><td>read.parquet.vectorization.batch-size</td><td>5000</td><td>The batch size for parquet vectorized reads</td></tr><tr><td>read.orc.vectorization.enabled</td><td>false</td><td>Controls whether orc vectorized reads are used</td></tr><tr><td>read.orc.vectorization.batch-size</td><td>5000</td><td>The batch size for orc vectorized reads</td></tr></tbody></table><h3 id=write-properties>Write properties</h3><table><thead><tr><th>Property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>write.format.default</td><td>parquet</td><td>Default file format for the table; parquet, avro, or orc</td></tr><tr><td>write.delete.format.default</td><td>data file format</td><td>Default delete file format for the table; parquet, avro, or orc</td></tr><tr><td>write.parquet.row-group-size-bytes</td><td>134217728 (128 MB)</td><td>Parquet row group size</td></tr><tr><td>write.parquet.page-size-bytes</td><td>1048576 (1 MB)</td><td>Parquet page size</td></tr><tr><td>write.parquet.page-row-limit</td><td>20000</td><td>Parquet page row limit</td></tr><tr><td>write.parquet.dict-size-bytes</td><td>2097152 (2 MB)</td><td>Parquet dictionary page size</td></tr><tr><td>write.parquet.compression-codec</td><td>gzip</td><td>Parquet compression codec: zstd, brotli, lz4, gzip, snappy, uncompressed</td></tr><tr><td>write.parquet.compression-level</td><td>null</td><td>Parquet compression level</td></tr><tr><td>write.parquet.bloom-filter-enabled.column.col1</td><td>(not set)</td><td>Hint to parquet to write a bloom filter for the column: col1</td></tr><tr><td>write.parquet.bloom-filter-max-bytes</td><td>1048576 (1 MB)</td><td>The maximum number of bytes for a bloom filter bitset</td></tr><tr><td>write.avro.compression-codec</td><td>gzip</td><td>Avro compression codec: gzip(deflate with 9 level), zstd, snappy, uncompressed</td></tr><tr><td>write.avro.compression-level</td><td>null</td><td>Avro compression level</td></tr><tr><td>write.orc.stripe-size-bytes</td><td>67108864 (64 MB)</td><td>Define the default ORC stripe size, in bytes</td></tr><tr><td>write.orc.block-size-bytes</td><td>268435456 (256 MB)</td><td>Define the default file system block size for ORC files</td></tr><tr><td>write.orc.compression-codec</td><td>zlib</td><td>ORC compression codec: zstd, lz4, lzo, zlib, snappy, none</td></tr><tr><td>write.orc.compression-strategy</td><td>speed</td><td>ORC compression strategy: speed, compression</td></tr><tr><td>write.orc.bloom.filter.columns</td><td>(not set)</td><td>Comma separated list of column names for which a Bloom filter must be created</td></tr><tr><td>write.orc.bloom.filter.fpp</td><td>0.05</td><td>False positive probability for Bloom filter (must > 0.0 and &lt; 1.0)</td></tr><tr><td>write.location-provider.impl</td><td>null</td><td>Optional custom implementation for LocationProvider</td></tr><tr><td>write.metadata.compression-codec</td><td>none</td><td>Metadata compression codec; none or gzip</td></tr><tr><td>write.metadata.metrics.max-inferred-column-defaults</td><td>100</td><td>Defines the maximum number of columns for which metrics are collected</td></tr><tr><td>write.metadata.metrics.default</td><td>truncate(16)</td><td>Default metrics mode for all columns in the table; none, counts, truncate(length), or full</td></tr><tr><td>write.metadata.metrics.column.col1</td><td>(not set)</td><td>Metrics mode for column &lsquo;col1&rsquo; to allow per-column tuning; none, counts, truncate(length), or full</td></tr><tr><td>write.target-file-size-bytes</td><td>536870912 (512 MB)</td><td>Controls the size of files generated to target about this many bytes</td></tr><tr><td>write.delete.target-file-size-bytes</td><td>67108864 (64 MB)</td><td>Controls the size of delete files generated to target about this many bytes</td></tr><tr><td>write.distribution-mode</td><td>none</td><td>Defines distribution of write data: <strong>none</strong>: don&rsquo;t shuffle rows; <strong>hash</strong>: hash distribute by partition key ; <strong>range</strong>: range distribute by partition key or sort key if table has an SortOrder</td></tr><tr><td>write.delete.distribution-mode</td><td>hash</td><td>Defines distribution of write delete data</td></tr><tr><td>write.update.distribution-mode</td><td>hash</td><td>Defines distribution of write update data</td></tr><tr><td>write.merge.distribution-mode</td><td>none</td><td>Defines distribution of write merge data</td></tr><tr><td>write.wap.enabled</td><td>false</td><td>Enables write-audit-publish writes</td></tr><tr><td>write.summary.partition-limit</td><td>0</td><td>Includes partition-level summary stats in snapshot summaries if the changed partition count is less than this limit</td></tr><tr><td>write.metadata.delete-after-commit.enabled</td><td>false</td><td>Controls whether to delete the oldest <strong>tracked</strong> version metadata files after commit</td></tr><tr><td>write.metadata.previous-versions-max</td><td>100</td><td>The max number of previous version metadata files to keep before deleting after commit</td></tr><tr><td>write.spark.fanout.enabled</td><td>false</td><td>Enables the fanout writer in Spark that does not require data to be clustered; uses more memory</td></tr><tr><td>write.object-storage.enabled</td><td>false</td><td>Enables the object storage location provider that adds a hash component to file paths</td></tr><tr><td>write.data.path</td><td>table location + /data</td><td>Base location for data files</td></tr><tr><td>write.metadata.path</td><td>table location + /metadata</td><td>Base location for metadata files</td></tr><tr><td>write.delete.mode</td><td>copy-on-write</td><td>Mode used for delete commands: copy-on-write or merge-on-read (v2 only)</td></tr><tr><td>write.delete.isolation-level</td><td>serializable</td><td>Isolation level for delete commands: serializable or snapshot</td></tr><tr><td>write.update.mode</td><td>copy-on-write</td><td>Mode used for update commands: copy-on-write or merge-on-read (v2 only)</td></tr><tr><td>write.update.isolation-level</td><td>serializable</td><td>Isolation level for update commands: serializable or snapshot</td></tr><tr><td>write.merge.mode</td><td>copy-on-write</td><td>Mode used for merge commands: copy-on-write or merge-on-read (v2 only)</td></tr><tr><td>write.merge.isolation-level</td><td>serializable</td><td>Isolation level for merge commands: serializable or snapshot</td></tr></tbody></table><h3 id=table-behavior-properties>Table behavior properties</h3><table><thead><tr><th>Property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>commit.retry.num-retries</td><td>4</td><td>Number of times to retry a commit before failing</td></tr><tr><td>commit.retry.min-wait-ms</td><td>100</td><td>Minimum time in milliseconds to wait before retrying a commit</td></tr><tr><td>commit.retry.max-wait-ms</td><td>60000 (1 min)</td><td>Maximum time in milliseconds to wait before retrying a commit</td></tr><tr><td>commit.retry.total-timeout-ms</td><td>1800000 (30 min)</td><td>Total retry timeout period in milliseconds for a commit</td></tr><tr><td>commit.status-check.num-retries</td><td>3</td><td>Number of times to check whether a commit succeeded after a connection is lost before failing due to an unknown commit state</td></tr><tr><td>commit.status-check.min-wait-ms</td><td>1000 (1s)</td><td>Minimum time in milliseconds to wait before retrying a status-check</td></tr><tr><td>commit.status-check.max-wait-ms</td><td>60000 (1 min)</td><td>Maximum time in milliseconds to wait before retrying a status-check</td></tr><tr><td>commit.status-check.total-timeout-ms</td><td>1800000 (30 min)</td><td>Total timeout period in which the commit status-check must succeed, in milliseconds</td></tr><tr><td>commit.manifest.target-size-bytes</td><td>8388608 (8 MB)</td><td>Target size when merging manifest files</td></tr><tr><td>commit.manifest.min-count-to-merge</td><td>100</td><td>Minimum number of manifests to accumulate before merging</td></tr><tr><td>commit.manifest-merge.enabled</td><td>true</td><td>Controls whether to automatically merge manifests on writes</td></tr><tr><td>history.expire.max-snapshot-age-ms</td><td>432000000 (5 days)</td><td>Default max age of snapshots to keep on the table and all of its branches while expiring snapshots</td></tr><tr><td>history.expire.min-snapshots-to-keep</td><td>1</td><td>Default min number of snapshots to keep on the table and all of its branches while expiring snapshots</td></tr><tr><td>history.expire.max-ref-age-ms</td><td><code>Long.MAX_VALUE</code> (forever)</td><td>For snapshot references except the <code>main</code> branch, default max age of snapshot references to keep while expiring snapshots. The <code>main</code> branch never expires.</td></tr></tbody></table><h3 id=reserved-table-properties>Reserved table properties</h3><p>Reserved table properties are only used to control behaviors when creating or updating a table.
The value of these properties are not persisted as a part of the table metadata.</p><table><thead><tr><th>Property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>format-version</td><td>1</td><td>Table&rsquo;s format version (can be 1 or 2) as defined in the <a href=../../../spec/#format-versioning>Spec</a>.</td></tr></tbody></table><h3 id=compatibility-flags>Compatibility flags</h3><table><thead><tr><th>Property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>compatibility.snapshot-id-inheritance.enabled</td><td>false</td><td>Enables committing snapshots without explicit snapshot IDs</td></tr></tbody></table><h2 id=catalog-properties>Catalog properties</h2><p>Iceberg catalogs support using catalog properties to configure catalog behaviors. Here is a list of commonly used catalog properties:</p><table><thead><tr><th>Property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>catalog-impl</td><td>null</td><td>a custom <code>Catalog</code> implementation to use by an engine</td></tr><tr><td>io-impl</td><td>null</td><td>a custom <code>FileIO</code> implementation to use in a catalog</td></tr><tr><td>warehouse</td><td>null</td><td>the root path of the data warehouse</td></tr><tr><td>uri</td><td>null</td><td>a URI string, such as Hive metastore URI</td></tr><tr><td>clients</td><td>2</td><td>client pool size</td></tr><tr><td>cache-enabled</td><td>true</td><td>Whether to cache catalog entries</td></tr><tr><td>cache.expiration-interval-ms</td><td>30000</td><td>How long catalog entries are locally cached, in milliseconds; 0 disables caching, negative values disable expiration</td></tr><tr><td>metrics-reporter-impl</td><td>org.apache.iceberg.metrics.LoggingMetricsReporter</td><td>Custom <code>MetricsReporter</code> implementation to use in a catalog. See the <a href=metrics-reporting>Metrics reporting</a> section for additional details</td></tr></tbody></table><p><code>HadoopCatalog</code> and <code>HiveCatalog</code> can access the properties in their constructors.
Any other custom catalog can access the properties by implementing <code>Catalog.initialize(catalogName, catalogProperties)</code>.
The properties can be manually constructed or passed in from a compute engine like Spark or Flink.
Spark uses its session properties as catalog properties, see more details in the <a href=../spark-configuration#catalog-configuration>Spark configuration</a> section.
Flink passes in catalog properties through <code>CREATE CATALOG</code> statement, see more details in the <a href=../flink/#creating-catalogs-and-using-catalogs>Flink</a> section.</p><h3 id=lock-catalog-properties>Lock catalog properties</h3><p>Here are the catalog properties related to locking. They are used by some catalog implementations to control the locking behavior during commits.</p><table><thead><tr><th>Property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>lock-impl</td><td>null</td><td>a custom implementation of the lock manager, the actual interface depends on the catalog used</td></tr><tr><td>lock.table</td><td>null</td><td>an auxiliary table for locking, such as in <a href=../aws/#dynamodb-for-commit-locking>AWS DynamoDB lock manager</a></td></tr><tr><td>lock.acquire-interval-ms</td><td>5000 (5 s)</td><td>the interval to wait between each attempt to acquire a lock</td></tr><tr><td>lock.acquire-timeout-ms</td><td>180000 (3 min)</td><td>the maximum time to try acquiring a lock</td></tr><tr><td>lock.heartbeat-interval-ms</td><td>3000 (3 s)</td><td>the interval to wait between each heartbeat after acquiring a lock</td></tr><tr><td>lock.heartbeat-timeout-ms</td><td>15000 (15 s)</td><td>the maximum time without a heartbeat to consider a lock expired</td></tr></tbody></table><h2 id=hadoop-configuration>Hadoop configuration</h2><p>The following properties from the Hadoop configuration are used by the Hive Metastore connector.
The HMS table locking is a 2-step process:</p><ol><li>Lock Creation: Create lock in HMS and queue for acquisition</li><li>Lock Check: Check if lock successfully acquired</li></ol><table><thead><tr><th>Property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>iceberg.hive.client-pool-size</td><td>5</td><td>The size of the Hive client pool when tracking tables in HMS</td></tr><tr><td>iceberg.hive.lock-creation-timeout-ms</td><td>180000 (3 min)</td><td>Maximum time in milliseconds to create a lock in the HMS</td></tr><tr><td>iceberg.hive.lock-creation-min-wait-ms</td><td>50</td><td>Minimum time in milliseconds between retries of creating the lock in the HMS</td></tr><tr><td>iceberg.hive.lock-creation-max-wait-ms</td><td>5000</td><td>Maximum time in milliseconds between retries of creating the lock in the HMS</td></tr><tr><td>iceberg.hive.lock-timeout-ms</td><td>180000 (3 min)</td><td>Maximum time in milliseconds to acquire a lock</td></tr><tr><td>iceberg.hive.lock-check-min-wait-ms</td><td>50</td><td>Minimum time in milliseconds between checking the acquisition of the lock</td></tr><tr><td>iceberg.hive.lock-check-max-wait-ms</td><td>5000</td><td>Maximum time in milliseconds between checking the acquisition of the lock</td></tr><tr><td>iceberg.hive.lock-heartbeat-interval-ms</td><td>240000 (4 min)</td><td>The heartbeat interval for the HMS locks.</td></tr><tr><td>iceberg.hive.metadata-refresh-max-retries</td><td>2</td><td>Maximum number of retries when the metadata file is missing</td></tr><tr><td>iceberg.hive.table-level-lock-evict-ms</td><td>600000 (10 min)</td><td>The timeout for the JVM table lock is</td></tr><tr><td>iceberg.engine.hive.lock-enabled</td><td>true</td><td>Use HMS locks to ensure atomicity of commits</td></tr></tbody></table><p>Note: <code>iceberg.hive.lock-check-max-wait-ms</code> and <code>iceberg.hive.lock-heartbeat-interval-ms</code> should be less than the <a href=https://cwiki.apache.org/confluence/display/Hive/Configuration+Properties#ConfigurationProperties-hive.txn.timeout>transaction timeout</a>
of the Hive Metastore (<code>hive.txn.timeout</code> or <code>metastore.txn.timeout</code> in the newer versions). Otherwise, the heartbeats on the lock (which happens during the lock checks) would end up expiring in the
Hive Metastore before the lock is retried from Iceberg.</p><p>Warn: Setting <code>iceberg.engine.hive.lock-enabled</code>=<code>false</code> will cause HiveCatalog to commit to tables without using Hive locks.
This should only be set to <code>false</code> if all following conditions are met:</p><ul><li><a href=https://issues.apache.org/jira/browse/HIVE-26882>HIVE-26882</a>
is available on the Hive Metastore server</li><li>All other HiveCatalogs committing to tables that this HiveCatalog commits to are also on Iceberg 1.3 or later</li><li>All other HiveCatalogs committing to tables that this HiveCatalog commits to have also disabled Hive locks on commit.</li></ul><p><strong>Failing to ensure these conditions risks corrupting the table.</strong></p><p>Even with <code>iceberg.engine.hive.lock-enabled</code> set to <code>false</code>, a HiveCatalog can still use locks for individual tables by setting the table property <code>engine.hive.lock-enabled</code>=<code>true</code>.
This is useful in the case where other HiveCatalogs cannot be upgraded and set to commit without using Hive locks.</p></div><div id=toc class=markdown-body><div id=full><nav id=TableOfContents><ul><li><a href=#table-properties>Table properties</a><ul><li><a href=#read-properties>Read properties</a></li><li><a href=#write-properties>Write properties</a></li><li><a href=#table-behavior-properties>Table behavior properties</a></li><li><a href=#reserved-table-properties>Reserved table properties</a></li><li><a href=#compatibility-flags>Compatibility flags</a></li></ul></li><li><a href=#catalog-properties>Catalog properties</a><ul><li><a href=#lock-catalog-properties>Lock catalog properties</a></li></ul></li><li><a href=#hadoop-configuration>Hadoop configuration</a></li></ul></nav></div></div></div></div></section></body><script src=https://iceberg.apache.org/docs/fd-update-slack-url//js/jquery-1.11.0.js></script><script src=https://iceberg.apache.org/docs/fd-update-slack-url//js/jquery.easing.min.js></script><script type=text/javascript src=https://iceberg.apache.org/docs/fd-update-slack-url//js/search.js></script><script src=https://iceberg.apache.org/docs/fd-update-slack-url//js/bootstrap.min.js></script><script src=https://iceberg.apache.org/docs/fd-update-slack-url//js/iceberg-theme.js></script></html>