| <!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>Flink 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-javadocs//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.1</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-javadocs/../latest>latest</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../1.4.1>1.4.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../1.4.0>1.4.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../1.3.1>1.3.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../1.3.0>1.3.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../1.2.1>1.2.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../1.2.0>1.2.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../1.1.0>1.1.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../1.0.0>1.0.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../0.14.1>0.14.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../0.14.0>0.14.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../0.13.2>0.13.2</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../0.13.1>0.13.1</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../0.13.0>0.13.0</a></li><li class=versions-dropdown-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../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-javadocs/../../hive-quickstart>Hive</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../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-javadocs/../../docs/latest>Docs</a></div><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../releases>Releases</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../roadmap>Roadmap</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../blogs>Blogs</a></div class="topnav-page-selection"><div class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../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-javadocs/../../community>Community</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../spec>Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../view-spec>View Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../puffin-spec>Puffin Spec</a></li class="topnav-page-selection"><li class=topnav-page-selection><a href=https://iceberg.apache.org/docs/fd-update-javadocs/../../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-javadocs/../../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-javadocs/../../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-javadocs/../../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-javadocs//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-2561tq9qr-UtISlHgsdY3Virs3Z2_btQ target=_blank><img src=https://iceberg.apache.org/docs/fd-update-javadocs//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 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 in"><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 id=active 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-javadocs/../../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=flink-configuration>Flink Configuration</h1><h2 id=catalog-configuration>Catalog Configuration</h2><p>A catalog is created and named by executing the following query (replace <code><catalog_name></code> with your catalog name and |
| <code><config_key></code>=<code><config_value></code> with catalog implementation config):</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>CREATE</span> <span style=color:#66d9ef>CATALOG</span> <span style=color:#f92672><</span><span style=color:#66d9ef>catalog_name</span><span style=color:#f92672>></span> <span style=color:#66d9ef>WITH</span> ( |
| </span></span><span style=display:flex><span> <span style=color:#e6db74>'type'</span><span style=color:#f92672>=</span><span style=color:#e6db74>'iceberg'</span>, |
| </span></span><span style=display:flex><span> <span style=color:#f92672>`<</span>config_key<span style=color:#f92672>>`=`<</span>config_value<span style=color:#f92672>>`</span> |
| </span></span><span style=display:flex><span>); |
| </span></span></code></pre></div><p>The following properties can be set globally and are not limited to a specific catalog implementation:</p><table><thead><tr><th>Property</th><th>Required</th><th>Values</th><th>Description</th></tr></thead><tbody><tr><td>type</td><td>✔️</td><td>iceberg</td><td>Must be <code>iceberg</code>.</td></tr><tr><td>catalog-type</td><td></td><td><code>hive</code>, <code>hadoop</code> or <code>rest</code></td><td><code>hive</code>, <code>hadoop</code> or <code>rest</code> for built-in catalogs, or left unset for custom catalog implementations using catalog-impl.</td></tr><tr><td>catalog-impl</td><td></td><td></td><td>The fully-qualified class name of a custom catalog implementation. Must be set if <code>catalog-type</code> is unset.</td></tr><tr><td>property-version</td><td></td><td></td><td>Version number to describe the property version. This property can be used for backwards compatibility in case the property format changes. The current property version is <code>1</code>.</td></tr><tr><td>cache-enabled</td><td></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>cache.expiration-interval-ms</td><td></td><td></td><td>How long catalog entries are locally cached, in milliseconds; negative values like <code>-1</code> will disable expiration, value 0 is not allowed to set. default value is <code>-1</code>.</td></tr></tbody></table><p>The following properties can be set if using the Hive catalog:</p><table><thead><tr><th>Property</th><th>Required</th><th>Values</th><th>Description</th></tr></thead><tbody><tr><td>uri</td><td>✔️</td><td></td><td>The Hive metastore’s thrift URI.</td></tr><tr><td>clients</td><td></td><td></td><td>The Hive metastore client pool size, default value is 2.</td></tr><tr><td>warehouse</td><td></td><td></td><td>The Hive warehouse location, users should specify this path if neither set the <code>hive-conf-dir</code> to specify a location containing a <code>hive-site.xml</code> configuration file nor add a correct <code>hive-site.xml</code> to classpath.</td></tr><tr><td>hive-conf-dir</td><td></td><td></td><td>Path to a directory containing a <code>hive-site.xml</code> configuration file which will be used to provide custom Hive configuration values. The value of <code>hive.metastore.warehouse.dir</code> from <code><hive-conf-dir>/hive-site.xml</code> (or hive configure file from classpath) will be overwritten with the <code>warehouse</code> value if setting both <code>hive-conf-dir</code> and <code>warehouse</code> when creating iceberg catalog.</td></tr><tr><td>hadoop-conf-dir</td><td></td><td></td><td>Path to a directory containing <code>core-site.xml</code> and <code>hdfs-site.xml</code> configuration files which will be used to provide custom Hadoop configuration values.</td></tr></tbody></table><p>The following properties can be set if using the Hadoop catalog:</p><table><thead><tr><th>Property</th><th>Required</th><th>Values</th><th>Description</th></tr></thead><tbody><tr><td>warehouse</td><td>✔️</td><td></td><td>The HDFS directory to store metadata files and data files.</td></tr></tbody></table><p>The following properties can be set if using the REST catalog:</p><table><thead><tr><th>Property</th><th>Required</th><th>Values</th><th>Description</th></tr></thead><tbody><tr><td>uri</td><td>✔️</td><td></td><td>The URL to the REST Catalog.</td></tr><tr><td>credential</td><td></td><td></td><td>A credential to exchange for a token in the OAuth2 client credentials flow.</td></tr><tr><td>token</td><td></td><td></td><td>A token which will be used to interact with the server.</td></tr></tbody></table><h2 id=runtime-configuration>Runtime configuration</h2><h3 id=read-options>Read options</h3><p>Flink read options are passed when configuring the Flink IcebergSource:</p><pre tabindex=0><code>IcebergSource.forRowData() |
| .tableLoader(TableLoader.fromCatalog(...)) |
| .assignerFactory(new SimpleSplitAssignerFactory()) |
| .streaming(true) |
| .streamingStartingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) |
| .startSnapshotId(3821550127947089987L) |
| .monitorInterval(Duration.ofMillis(10L)) // or .set("monitor-interval", "10s") \ set(FlinkReadOptions.MONITOR_INTERVAL, "10s") |
| .build() |
| </code></pre><p>For Flink SQL, read options can be passed in via SQL hints like this:</p><pre tabindex=0><code>SELECT * FROM tableName /*+ OPTIONS('monitor-interval'='10s') */ |
| ... |
| </code></pre><p>Options can be passed in via Flink configuration, which will be applied to current session. Note that not all options support this mode.</p><pre tabindex=0><code>env.getConfig() |
| .getConfiguration() |
| .set(FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION, 1000L); |
| ... |
| </code></pre><p><code>Read option</code> has the highest priority, followed by <code>Flink configuration</code> and then <code>Table property</code>.</p><table><thead><tr><th>Read option</th><th>Flink configuration</th><th>Table property</th><th>Default</th><th>Description</th></tr></thead><tbody><tr><td>snapshot-id</td><td>N/A</td><td>N/A</td><td>null</td><td>For time travel in batch mode. Read data from the specified snapshot-id.</td></tr><tr><td>case-sensitive</td><td>connector.iceberg.case-sensitive</td><td>N/A</td><td>false</td><td>If true, match column name in a case sensitive way.</td></tr><tr><td>as-of-timestamp</td><td>N/A</td><td>N/A</td><td>null</td><td>For time travel in batch mode. Read data from the most recent snapshot as of the given time in milliseconds.</td></tr><tr><td>starting-strategy</td><td>connector.iceberg.starting-strategy</td><td>N/A</td><td>INCREMENTAL_FROM_LATEST_SNAPSHOT</td><td>Starting strategy for streaming execution. TABLE_SCAN_THEN_INCREMENTAL: Do a regular table scan then switch to the incremental mode. The incremental mode starts from the current snapshot exclusive. INCREMENTAL_FROM_LATEST_SNAPSHOT: Start incremental mode from the latest snapshot inclusive. If it is an empty map, all future append snapshots should be discovered. INCREMENTAL_FROM_EARLIEST_SNAPSHOT: Start incremental mode from the earliest snapshot inclusive. If it is an empty map, all future append snapshots should be discovered. INCREMENTAL_FROM_SNAPSHOT_ID: Start incremental mode from a snapshot with a specific id inclusive. INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: Start incremental mode from a snapshot with a specific timestamp inclusive. If the timestamp is between two snapshots, it should start from the snapshot after the timestamp. Just for FIP27 Source.</td></tr><tr><td>start-snapshot-timestamp</td><td>N/A</td><td>N/A</td><td>null</td><td>Start to read data from the most recent snapshot as of the given time in milliseconds.</td></tr><tr><td>start-snapshot-id</td><td>N/A</td><td>N/A</td><td>null</td><td>Start to read data from the specified snapshot-id.</td></tr><tr><td>end-snapshot-id</td><td>N/A</td><td>N/A</td><td>The latest snapshot id</td><td>Specifies the end snapshot.</td></tr><tr><td>branch</td><td>N/A</td><td>N/A</td><td>main</td><td>Specifies the branch to read from in batch mode</td></tr><tr><td>tag</td><td>N/A</td><td>N/A</td><td>null</td><td>Specifies the tag to read from in batch mode</td></tr><tr><td>start-tag</td><td>N/A</td><td>N/A</td><td>null</td><td>Specifies the starting tag to read from for incremental reads</td></tr><tr><td>end-tag</td><td>N/A</td><td>N/A</td><td>null</td><td>Specifies the ending tag to to read from for incremental reads</td></tr><tr><td>split-size</td><td>connector.iceberg.split-size</td><td>read.split.target-size</td><td>128 MB</td><td>Target size when combining input splits.</td></tr><tr><td>split-lookback</td><td>connector.iceberg.split-file-open-cost</td><td>read.split.planning-lookback</td><td>10</td><td>Number of bins to consider when combining input splits.</td></tr><tr><td>split-file-open-cost</td><td>connector.iceberg.split-file-open-cost</td><td>read.split.open-file-cost</td><td>4MB</td><td>The estimated cost to open a file, used as a minimum weight when combining splits.</td></tr><tr><td>streaming</td><td>connector.iceberg.streaming</td><td>N/A</td><td>false</td><td>Sets whether the current task runs in streaming or batch mode.</td></tr><tr><td>monitor-interval</td><td>connector.iceberg.monitor-interval</td><td>N/A</td><td>60s</td><td>Monitor interval to discover splits from new snapshots. Applicable only for streaming read.</td></tr><tr><td>include-column-stats</td><td>connector.iceberg.include-column-stats</td><td>N/A</td><td>false</td><td>Create a new scan from this that loads the column stats with each data file. Column stats include: value count, null value count, lower bounds, and upper bounds.</td></tr><tr><td>max-planning-snapshot-count</td><td>connector.iceberg.max-planning-snapshot-count</td><td>N/A</td><td>Integer.MAX_VALUE</td><td>Max number of snapshots limited per split enumeration. Applicable only to streaming read.</td></tr><tr><td>limit</td><td>connector.iceberg.limit</td><td>N/A</td><td>-1</td><td>Limited output number of rows.</td></tr></tbody></table><h3 id=write-options>Write options</h3><p>Flink write options are passed when configuring the FlinkSink, like this:</p><pre tabindex=0><code>FlinkSink.Builder builder = FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) |
| .table(table) |
| .tableLoader(tableLoader) |
| .set("write-format", "orc") |
| .set(FlinkWriteOptions.OVERWRITE_MODE, "true"); |
| </code></pre><p>For Flink SQL, write options can be passed in via SQL hints like this:</p><pre tabindex=0><code>INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ |
| ... |
| </code></pre><table><thead><tr><th>Flink 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’s write.target-file-size-bytes</td></tr><tr><td>upsert-enabled</td><td>Table write.upsert.enabled</td><td>Overrides this table’s write.upsert.enabled</td></tr><tr><td>overwrite-enabled</td><td>false</td><td>Overwrite the table’s data, overwrite mode shouldn’t be enable when configuring to use UPSERT data stream.</td></tr><tr><td>distribution-mode</td><td>Table write.distribution-mode</td><td>Overrides this table’s write.distribution-mode</td></tr><tr><td>compression-codec</td><td>Table write.(fileformat).compression-codec</td><td>Overrides this table’s compression codec for this write</td></tr><tr><td>compression-level</td><td>Table write.(fileformat).compression-level</td><td>Overrides this table’s compression level for Parquet and Avro tables for this write</td></tr><tr><td>compression-strategy</td><td>Table write.orc.compression-strategy</td><td>Overrides this table’s compression strategy for ORC tables for this write</td></tr><tr><td>write-parallelism</td><td>Upstream operator parallelism</td><td>Overrides the writer parallelism</td></tr></tbody></table></div><div id=toc class=markdown-body><div id=full><nav id=TableOfContents><ul><li><a href=#catalog-configuration>Catalog Configuration</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-update-javadocs//js/jquery-1.11.0.js></script> |
| <script src=https://iceberg.apache.org/docs/fd-update-javadocs//js/jquery.easing.min.js></script> |
| <script type=text/javascript src=https://iceberg.apache.org/docs/fd-update-javadocs//js/search.js></script> |
| <script src=https://iceberg.apache.org/docs/fd-update-javadocs//js/bootstrap.min.js></script> |
| <script src=https://iceberg.apache.org/docs/fd-update-javadocs//js/iceberg-theme.js></script></html> |