blob: 5bcadc96457ef2521f23c0905e4961aeffe0b957 [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>Flink DDL</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 id=active 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-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><h2 id=ddl-commands>DDL commands</h2><h3 id=create-catalog><code>CREATE Catalog</code></h3><h4 id=hive-catalog>Hive catalog</h4><p>This creates an Iceberg catalog named <code>hive_catalog</code> that can be configured using <code>'catalog-type'='hive'</code>, which loads tables from 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-sql data-lang=sql><span style=display:flex><span><span style=color:#66d9ef>CREATE</span> <span style=color:#66d9ef>CATALOG</span> hive_catalog <span style=color:#66d9ef>WITH</span> (
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;iceberg&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;catalog-type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;hive&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;uri&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;thrift://localhost:9083&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;clients&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;5&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;property-version&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;1&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;warehouse&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;hdfs://nn:8020/warehouse/path&#39;</span>
</span></span><span style=display:flex><span>);
</span></span></code></pre></div><p>The following properties can be set if using the Hive catalog:</p><ul><li><code>uri</code>: The Hive metastore&rsquo;s thrift URI. (Required)</li><li><code>clients</code>: The Hive metastore client pool size, default value is 2. (Optional)</li><li><code>warehouse</code>: 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.</li><li><code>hive-conf-dir</code>: 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>&lt;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.</li><li><code>hadoop-conf-dir</code>: 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.</li></ul><h4 id=hadoop-catalog>Hadoop catalog</h4><p>Iceberg also supports a directory-based catalog in HDFS that can be configured using <code>'catalog-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-sql data-lang=sql><span style=display:flex><span><span style=color:#66d9ef>CREATE</span> <span style=color:#66d9ef>CATALOG</span> hadoop_catalog <span style=color:#66d9ef>WITH</span> (
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;iceberg&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;catalog-type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;hadoop&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;warehouse&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;hdfs://nn:8020/warehouse/path&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;property-version&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;1&#39;</span>
</span></span><span style=display:flex><span>);
</span></span></code></pre></div><p>The following properties can be set if using the Hadoop catalog:</p><ul><li><code>warehouse</code>: The HDFS directory to store metadata files and data files. (Required)</li></ul><p>Execute the sql command <code>USE CATALOG hadoop_catalog</code> to set the current catalog.</p><h4 id=rest-catalog>REST catalog</h4><p>This creates an iceberg catalog named <code>rest_catalog</code> that can be configured using <code>'catalog-type'='rest'</code>, which loads tables from a REST catalog:</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> rest_catalog <span style=color:#66d9ef>WITH</span> (
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;iceberg&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;catalog-type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;rest&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;uri&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;https://localhost/&#39;</span>
</span></span><span style=display:flex><span>);
</span></span></code></pre></div><p>The following properties can be set if using the REST catalog:</p><ul><li><code>uri</code>: The URL to the REST Catalog (Required)</li><li><code>credential</code>: A credential to exchange for a token in the OAuth2 client credentials flow (Optional)</li><li><code>token</code>: A token which will be used to interact with the server (Optional)</li></ul><h4 id=custom-catalog>Custom catalog</h4><p>Flink also supports loading a custom Iceberg <code>Catalog</code> implementation by specifying the <code>catalog-impl</code> property:</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> my_catalog <span style=color:#66d9ef>WITH</span> (
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;iceberg&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;catalog-impl&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;com.my.custom.CatalogImpl&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;my-additional-catalog-config&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;my-value&#39;</span>
</span></span><span style=display:flex><span>);
</span></span></code></pre></div><h4 id=create-through-yaml-config>Create through YAML config</h4><p>Catalogs can be registered in <code>sql-client-defaults.yaml</code> before starting the SQL client.</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-yaml data-lang=yaml><span style=display:flex><span><span style=color:#f92672>catalogs</span>:
</span></span><span style=display:flex><span> - <span style=color:#f92672>name</span>: <span style=color:#ae81ff>my_catalog</span>
</span></span><span style=display:flex><span> <span style=color:#f92672>type</span>: <span style=color:#ae81ff>iceberg</span>
</span></span><span style=display:flex><span> <span style=color:#f92672>catalog-type</span>: <span style=color:#ae81ff>hadoop</span>
</span></span><span style=display:flex><span> <span style=color:#f92672>warehouse</span>: <span style=color:#ae81ff>hdfs://nn:8020/warehouse/path</span>
</span></span></code></pre></div><h4 id=create-through-sql-files>Create through SQL Files</h4><p>The Flink SQL Client supports the <code>-i</code> startup option to execute an initialization SQL file to set up environment when starting up the SQL Client.</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:#75715e>-- define available catalogs
</span></span></span><span style=display:flex><span><span style=color:#75715e></span><span style=color:#66d9ef>CREATE</span> <span style=color:#66d9ef>CATALOG</span> hive_catalog <span style=color:#66d9ef>WITH</span> (
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;iceberg&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;catalog-type&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;hive&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;uri&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;thrift://localhost:9083&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#e6db74>&#39;warehouse&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;hdfs://nn:8020/warehouse/path&#39;</span>
</span></span><span style=display:flex><span>);
</span></span><span style=display:flex><span>
</span></span><span style=display:flex><span>USE <span style=color:#66d9ef>CATALOG</span> hive_catalog;
</span></span></code></pre></div><p>Using <code>-i &lt;init.sql></code> option to initialize SQL Client session:</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-bash data-lang=bash><span style=display:flex><span>/path/to/bin/sql-client.sh -i /path/to/init.sql
</span></span></code></pre></div><h3 id=create-database><code>CREATE DATABASE</code></h3><p>By default, Iceberg will use the <code>default</code> database in Flink. Using the following example to create a separate database in order to avoid creating tables under the <code>default</code> database:</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>DATABASE</span> iceberg_db;
</span></span><span style=display:flex><span>USE iceberg_db;
</span></span></code></pre></div><h3 id=create-table><code>CREATE TABLE</code></h3><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>TABLE</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>sample<span style=color:#f92672>`</span> (
</span></span><span style=display:flex><span> id BIGINT <span style=color:#66d9ef>COMMENT</span> <span style=color:#e6db74>&#39;unique id&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#66d9ef>data</span> STRING
</span></span><span style=display:flex><span>);
</span></span></code></pre></div><p>Table create commands support the commonly used <a href=https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sql/create/>Flink create clauses</a> including:</p><ul><li><code>PARTITION BY (column1, column2, ...)</code> to configure partitioning, Flink does not yet support hidden partitioning.</li><li><code>COMMENT 'table document'</code> to set a table description.</li><li><code>WITH ('key'='value', ...)</code> to set <a href=../configuration>table configuration</a> which will be stored in Iceberg table properties.</li></ul><p>Currently, it does not support computed column, primary key and watermark definition etc.</p><h3 id=partitioned-by><code>PARTITIONED BY</code></h3><p>To create a partition table, use <code>PARTITIONED BY</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-sql data-lang=sql><span style=display:flex><span><span style=color:#66d9ef>CREATE</span> <span style=color:#66d9ef>TABLE</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>sample<span style=color:#f92672>`</span> (
</span></span><span style=display:flex><span> id BIGINT <span style=color:#66d9ef>COMMENT</span> <span style=color:#e6db74>&#39;unique id&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#66d9ef>data</span> STRING
</span></span><span style=display:flex><span>) PARTITIONED <span style=color:#66d9ef>BY</span> (<span style=color:#66d9ef>data</span>);
</span></span></code></pre></div><p>Iceberg support hidden partition but Flink don&rsquo;t support partitioning by a function on columns, so there is no way to support hidden partition in Flink DDL.</p><h3 id=create-table-like><code>CREATE TABLE LIKE</code></h3><p>To create a table with the same schema, partitioning, and table properties as another table, use <code>CREATE TABLE LIKE</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-sql data-lang=sql><span style=display:flex><span><span style=color:#66d9ef>CREATE</span> <span style=color:#66d9ef>TABLE</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>sample<span style=color:#f92672>`</span> (
</span></span><span style=display:flex><span> id BIGINT <span style=color:#66d9ef>COMMENT</span> <span style=color:#e6db74>&#39;unique id&#39;</span>,
</span></span><span style=display:flex><span> <span style=color:#66d9ef>data</span> STRING
</span></span><span style=display:flex><span>);
</span></span><span style=display:flex><span>
</span></span><span style=display:flex><span><span style=color:#66d9ef>CREATE</span> <span style=color:#66d9ef>TABLE</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>sample_like<span style=color:#f92672>`</span> <span style=color:#66d9ef>LIKE</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>sample<span style=color:#f92672>`</span>;
</span></span></code></pre></div><p>For more details, refer to the <a href=https://nightlies.apache.org/flink/flink-docs-release-1.16/docs/dev/table/sql/create/>Flink <code>CREATE TABLE</code> documentation</a>.</p><h3 id=alter-table><code>ALTER TABLE</code></h3><p>Iceberg only support altering table properties:</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>ALTER</span> <span style=color:#66d9ef>TABLE</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>sample<span style=color:#f92672>`</span> <span style=color:#66d9ef>SET</span> (<span style=color:#e6db74>&#39;write.format.default&#39;</span><span style=color:#f92672>=</span><span style=color:#e6db74>&#39;avro&#39;</span>)
</span></span></code></pre></div><h3 id=alter-table--rename-to><code>ALTER TABLE .. RENAME TO</code></h3><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>ALTER</span> <span style=color:#66d9ef>TABLE</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>sample<span style=color:#f92672>`</span> <span style=color:#66d9ef>RENAME</span> <span style=color:#66d9ef>TO</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>new_sample<span style=color:#f92672>`</span>;
</span></span></code></pre></div><h3 id=drop-table><code>DROP TABLE</code></h3><p>To delete a table, run:</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>DROP</span> <span style=color:#66d9ef>TABLE</span> <span style=color:#f92672>`</span>hive_catalog<span style=color:#f92672>`</span>.<span style=color:#f92672>`</span><span style=color:#66d9ef>default</span><span style=color:#f92672>`</span>.<span style=color:#f92672>`</span>sample<span style=color:#f92672>`</span>;
</span></span></code></pre></div></div><div id=toc class=markdown-body><div id=full><nav id=TableOfContents><ul><li><a href=#ddl-commands>DDL commands</a><ul><li><a href=#create-catalog><code>CREATE Catalog</code></a></li><li><a href=#create-database><code>CREATE DATABASE</code></a></li><li><a href=#create-table><code>CREATE TABLE</code></a></li><li><a href=#partitioned-by><code>PARTITIONED BY</code></a></li><li><a href=#create-table-like><code>CREATE TABLE LIKE</code></a></li><li><a href=#alter-table><code>ALTER TABLE</code></a></li><li><a href=#alter-table--rename-to><code>ALTER TABLE .. RENAME TO</code></a></li><li><a href=#drop-table><code>DROP TABLE</code></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>