blob: 4980aec5be908df2a47ae98277bb7be115da142a [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-add-github-release//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.0</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-add-github-release/../latest>latest</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../1.4.0>1.4.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../1.3.1>1.3.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../1.3.0>1.3.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../1.2.1>1.2.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../1.2.0>1.2.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../1.1.0>1.1.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../1.0.0>1.0.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../0.14.1>0.14.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../0.14.0>0.14.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../0.13.2>0.13.2</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../0.13.1>0.13.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../0.13.0>0.13.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../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-add-github-release/../../hive-quickstart>Hive</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../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-add-github-release/../../docs/latest>Docs</a></div><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../releases>Releases</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../roadmap>Roadmap</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../blogs>Blogs</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../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-add-github-release/../../community>Community</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../spec>Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../view-spec>View Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../puffin-spec>Puffin Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-add-github-release/../../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-add-github-release/../../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-add-github-release/../../terms>Terms</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-add-github-release//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-1znkcg5zm-7_FE~pcox347XwZE3GNfPg target=_blank><img src=https://iceberg.apache.org/docs/fd-add-github-release//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 collapsed" 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><ul class=sub-menu><li><a href=../branching/>Branching and Tagging</a></li><li><a href=../configuration/>Configuration</a></li><li><a href=../evolution/>Evolution</a></li><li><a href=../maintenance/>Maintenance</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-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-add-github-release/../../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=spark-configuration>Spark Configuration</h1><h2 id=catalogs>Catalogs</h2><p>Spark adds an API to plug in table catalogs that are used to load, create, and manage Iceberg tables. Spark catalogs are configured by setting Spark properties under <code>spark.sql.catalog</code>.</p><p>This creates an Iceberg catalog named <code>hive_prod</code> that loads tables from a Hive metastore:</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-plain data-lang=plain><span style=display:flex><span>spark.sql.catalog.hive_prod = org.apache.iceberg.spark.SparkCatalog
</span></span><span style=display:flex><span>spark.sql.catalog.hive_prod.type = hive
</span></span><span style=display:flex><span>spark.sql.catalog.hive_prod.uri = thrift://metastore-host:port
</span></span><span style=display:flex><span># omit uri to use the same URI as Spark: hive.metastore.uris in hive-site.xml
</span></span></code></pre></div><p>Iceberg also supports a directory-based catalog in HDFS that can be configured using <code>type=hadoop</code>:</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-plain data-lang=plain><span style=display:flex><span>spark.sql.catalog.hadoop_prod = org.apache.iceberg.spark.SparkCatalog
</span></span><span style=display:flex><span>spark.sql.catalog.hadoop_prod.type = hadoop
</span></span><span style=display:flex><span>spark.sql.catalog.hadoop_prod.warehouse = hdfs://nn:8020/warehouse/path
</span></span></code></pre></div><div class=info>The Hive-based catalog only loads Iceberg tables. To load non-Iceberg tables in the same Hive metastore, use a <a href=#replacing-the-session-catalog>session catalog</a>.</div><h3 id=catalog-configuration>Catalog configuration</h3><p>A catalog is created and named by adding a property <code>spark.sql.catalog.(catalog-name)</code> with an implementation class for its value.</p><p>Iceberg supplies two implementations:</p><ul><li><code>org.apache.iceberg.spark.SparkCatalog</code> supports a Hive Metastore or a Hadoop warehouse as a catalog</li><li><code>org.apache.iceberg.spark.SparkSessionCatalog</code> adds support for Iceberg tables to Spark&rsquo;s built-in catalog, and delegates to the built-in catalog for non-Iceberg tables</li></ul><p>Both catalogs are configured using properties nested under the catalog name. Common configuration properties for Hive and Hadoop are:</p><table><thead><tr><th>Property</th><th>Values</th><th>Description</th></tr></thead><tbody><tr><td>spark.sql.catalog.<em>catalog-name</em>.type</td><td><code>hive</code>, <code>hadoop</code> or <code>rest</code></td><td>The underlying Iceberg catalog implementation, <code>HiveCatalog</code>, <code>HadoopCatalog</code>, <code>RESTCatalog</code> or left unset if using a custom catalog</td></tr><tr><td>spark.sql.catalog.<em>catalog-name</em>.catalog-impl</td><td></td><td>The underlying Iceberg catalog implementation.</td></tr><tr><td>spark.sql.catalog.<em>catalog-name</em>.default-namespace</td><td>default</td><td>The default current namespace for the catalog</td></tr><tr><td>spark.sql.catalog.<em>catalog-name</em>.uri</td><td>thrift://host:port</td><td>Metastore connect URI; default from <code>hive-site.xml</code></td></tr><tr><td>spark.sql.catalog.<em>catalog-name</em>.warehouse</td><td>hdfs://nn:8020/warehouse/path</td><td>Base path for the warehouse directory</td></tr><tr><td>spark.sql.catalog.<em>catalog-name</em>.cache-enabled</td><td><code>true</code> or <code>false</code></td><td>Whether to enable catalog cache, default value is <code>true</code></td></tr><tr><td>spark.sql.catalog.<em>catalog-name</em>.cache.expiration-interval-ms</td><td><code>30000</code> (30 seconds)</td><td>Duration after which cached catalog entries are expired; Only effective if <code>cache-enabled</code> is <code>true</code>. <code>-1</code> disables cache expiration and <code>0</code> disables caching entirely, irrespective of <code>cache-enabled</code>. Default is <code>30000</code> (30 seconds)</td></tr></tbody></table><p>Additional properties can be found in common <a href=../configuration#catalog-properties>catalog configuration</a>.</p><h3 id=using-catalogs>Using catalogs</h3><p>Catalog names are used in SQL queries to identify a table. In the examples above, <code>hive_prod</code> and <code>hadoop_prod</code> can be used to prefix database and table names that will be loaded from those catalogs.</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-sql data-lang=sql><span style=display:flex><span><span style=color:#66d9ef>SELECT</span> <span style=color:#f92672>*</span> <span style=color:#66d9ef>FROM</span> hive_prod.db.<span style=color:#66d9ef>table</span> <span style=color:#75715e>-- load db.table from catalog hive_prod
</span></span></span></code></pre></div><p>Spark 3 keeps track of the current catalog and namespace, which can be omitted from table names.</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-sql data-lang=sql><span style=display:flex><span>USE hive_prod.db;
</span></span><span style=display:flex><span><span style=color:#66d9ef>SELECT</span> <span style=color:#f92672>*</span> <span style=color:#66d9ef>FROM</span> <span style=color:#66d9ef>table</span> <span style=color:#75715e>-- load db.table from catalog hive_prod
</span></span></span></code></pre></div><p>To see the current catalog and namespace, run <code>SHOW CURRENT NAMESPACE</code>.</p><h3 id=replacing-the-session-catalog>Replacing the session catalog</h3><p>To add Iceberg table support to Spark&rsquo;s built-in catalog, configure <code>spark_catalog</code> to use Iceberg&rsquo;s <code>SparkSessionCatalog</code>.</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-plain data-lang=plain><span style=display:flex><span>spark.sql.catalog.spark_catalog = org.apache.iceberg.spark.SparkSessionCatalog
</span></span><span style=display:flex><span>spark.sql.catalog.spark_catalog.type = hive
</span></span></code></pre></div><p>Spark&rsquo;s built-in catalog supports existing v1 and v2 tables tracked in a Hive Metastore. This configures Spark to use Iceberg&rsquo;s <code>SparkSessionCatalog</code> as a wrapper around that session catalog. When a table is not an Iceberg table, the built-in catalog will be used to load it instead.</p><p>This configuration can use same Hive Metastore for both Iceberg and non-Iceberg tables.</p><h3 id=using-catalog-specific-hadoop-configuration-values>Using catalog specific Hadoop configuration values</h3><p>Similar to configuring Hadoop properties by using <code>spark.hadoop.*</code>, it&rsquo;s possible to set per-catalog Hadoop configuration values when using Spark by adding the property for the catalog with the prefix <code>spark.sql.catalog.(catalog-name).hadoop.*</code>. These properties will take precedence over values configured globally using <code>spark.hadoop.*</code> and will only affect Iceberg tables.</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-plain data-lang=plain><span style=display:flex><span>spark.sql.catalog.hadoop_prod.hadoop.fs.s3a.endpoint = http://aws-local:9000
</span></span></code></pre></div><h3 id=loading-a-custom-catalog>Loading a custom catalog</h3><p>Spark supports loading a custom Iceberg <code>Catalog</code> implementation by specifying the <code>catalog-impl</code> property. Here is an example:</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-plain data-lang=plain><span style=display:flex><span>spark.sql.catalog.custom_prod = org.apache.iceberg.spark.SparkCatalog
</span></span><span style=display:flex><span>spark.sql.catalog.custom_prod.catalog-impl = com.my.custom.CatalogImpl
</span></span><span style=display:flex><span>spark.sql.catalog.custom_prod.my-additional-catalog-config = my-value
</span></span></code></pre></div><h2 id=sql-extensions>SQL Extensions</h2><p>Iceberg 0.11.0 and later add an extension module to Spark to add new SQL commands, like <code>CALL</code> for stored procedures or <code>ALTER TABLE ... WRITE ORDERED BY</code>.</p><p>Using those SQL commands requires adding Iceberg extensions to your Spark environment using the following Spark property:</p><table><thead><tr><th>Spark extensions property</th><th>Iceberg extensions implementation</th></tr></thead><tbody><tr><td><code>spark.sql.extensions</code></td><td><code>org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions</code></td></tr></tbody></table><h2 id=runtime-configuration>Runtime configuration</h2><h3 id=read-options>Read options</h3><p>Spark read options are passed when configuring the DataFrameReader, like this:</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-scala data-lang=scala><span style=display:flex><span><span style=color:#75715e>// time travel
</span></span></span><span style=display:flex><span><span style=color:#75715e></span>spark<span style=color:#f92672>.</span>read
</span></span><span style=display:flex><span> <span style=color:#f92672>.</span>option<span style=color:#f92672>(</span><span style=color:#e6db74>&#34;snapshot-id&#34;</span><span style=color:#f92672>,</span> <span style=color:#ae81ff>10963874102873L</span><span style=color:#f92672>)</span>
</span></span><span style=display:flex><span> <span style=color:#f92672>.</span>table<span style=color:#f92672>(</span><span style=color:#e6db74>&#34;catalog.db.table&#34;</span><span style=color:#f92672>)</span>
</span></span></code></pre></div><table><thead><tr><th>Spark option</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>snapshot-id</td><td>(latest)</td><td>Snapshot ID of the table snapshot to read</td></tr><tr><td>as-of-timestamp</td><td>(latest)</td><td>A timestamp in milliseconds; the snapshot used will be the snapshot current at this time.</td></tr><tr><td>split-size</td><td>As per table property</td><td>Overrides this table&rsquo;s read.split.target-size and read.split.metadata-target-size</td></tr><tr><td>lookback</td><td>As per table property</td><td>Overrides this table&rsquo;s read.split.planning-lookback</td></tr><tr><td>file-open-cost</td><td>As per table property</td><td>Overrides this table&rsquo;s read.split.open-file-cost</td></tr><tr><td>vectorization-enabled</td><td>As per table property</td><td>Overrides this table&rsquo;s read.parquet.vectorization.enabled</td></tr><tr><td>batch-size</td><td>As per table property</td><td>Overrides this table&rsquo;s read.parquet.vectorization.batch-size</td></tr><tr><td>stream-from-timestamp</td><td>(none)</td><td>A timestamp in milliseconds to stream from; if before the oldest known ancestor snapshot, the oldest will be used</td></tr></tbody></table><h3 id=write-options>Write options</h3><p>Spark write options are passed when configuring the DataFrameWriter, like this:</p><div class=highlight><pre tabindex=0 style=color:#f8f8f2;background-color:#272822;-moz-tab-size:4;-o-tab-size:4;tab-size:4><code class=language-scala data-lang=scala><span style=display:flex><span><span style=color:#75715e>// write with Avro instead of Parquet
</span></span></span><span style=display:flex><span><span style=color:#75715e></span>df<span style=color:#f92672>.</span>write
</span></span><span style=display:flex><span> <span style=color:#f92672>.</span>option<span style=color:#f92672>(</span><span style=color:#e6db74>&#34;write-format&#34;</span><span style=color:#f92672>,</span> <span style=color:#e6db74>&#34;avro&#34;</span><span style=color:#f92672>)</span>
</span></span><span style=display:flex><span> <span style=color:#f92672>.</span>option<span style=color:#f92672>(</span><span style=color:#e6db74>&#34;snapshot-property.key&#34;</span><span style=color:#f92672>,</span> <span style=color:#e6db74>&#34;value&#34;</span><span style=color:#f92672>)</span>
</span></span><span style=display:flex><span> <span style=color:#f92672>.</span>insertInto<span style=color:#f92672>(</span><span style=color:#e6db74>&#34;catalog.db.table&#34;</span><span style=color:#f92672>)</span>
</span></span></code></pre></div><table><thead><tr><th>Spark option</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>write-format</td><td>Table write.format.default</td><td>File format to use for this write operation; parquet, avro, or orc</td></tr><tr><td>target-file-size-bytes</td><td>As per table property</td><td>Overrides this table&rsquo;s write.target-file-size-bytes</td></tr><tr><td>check-nullability</td><td>true</td><td>Sets the nullable check on fields</td></tr><tr><td>snapshot-property.<em>custom-key</em></td><td>null</td><td>Adds an entry with custom-key and corresponding value in the snapshot summary</td></tr><tr><td>fanout-enabled</td><td>false</td><td>Overrides this table&rsquo;s write.spark.fanout.enabled</td></tr><tr><td>check-ordering</td><td>true</td><td>Checks if input schema and table schema are same</td></tr><tr><td>isolation-level</td><td>null</td><td>Desired isolation level for Dataframe overwrite operations. <code>null</code> => no checks (for idempotent writes), <code>serializable</code> => check for concurrent inserts or deletes in destination partitions, <code>snapshot</code> => checks for concurrent deletes in destination partitions.</td></tr><tr><td>validate-from-snapshot-id</td><td>null</td><td>If isolation level is set, id of base snapshot from which to check concurrent write conflicts into a table. Should be the snapshot before any reads from the table. Can be obtained via <a href=../../api#table-metadata>Table API</a> or <a href=../spark-queries#snapshots>Snapshots table</a>. If null, the table&rsquo;s oldest known snapshot is used.</td></tr></tbody></table></div><div id=toc class=markdown-body><div id=full><nav id=TableOfContents><ul><li><a href=#catalogs>Catalogs</a><ul><li><a href=#catalog-configuration>Catalog configuration</a></li><li><a href=#using-catalogs>Using catalogs</a></li><li><a href=#replacing-the-session-catalog>Replacing the session catalog</a></li><li><a href=#using-catalog-specific-hadoop-configuration-values>Using catalog specific Hadoop configuration values</a></li><li><a href=#loading-a-custom-catalog>Loading a custom catalog</a></li></ul></li><li><a href=#sql-extensions>SQL Extensions</a></li><li><a href=#runtime-configuration>Runtime configuration</a><ul><li><a href=#read-options>Read options</a></li><li><a href=#write-options>Write options</a></li></ul></li></ul></nav></div></div></div></div></section></body><script src=https://iceberg.apache.org/docs/fd-add-github-release//js/jquery-1.11.0.js></script>
<script src=https://iceberg.apache.org/docs/fd-add-github-release//js/jquery.easing.min.js></script>
<script type=text/javascript src=https://iceberg.apache.org/docs/fd-add-github-release//js/search.js></script>
<script src=https://iceberg.apache.org/docs/fd-add-github-release//js/bootstrap.min.js></script>
<script src=https://iceberg.apache.org/docs/fd-add-github-release//js/iceberg-theme.js></script></html>