blob: ede5f84034bf629acdd6aa6750581a6834d1e443 [file] [log] [blame]
<!DOCTYPE html>
<html lang="en" dir=ZgotmplZ>
<head>
<link rel="stylesheet" href="/bootstrap/css/bootstrap.min.css">
<script src="/bootstrap/js/bootstrap.bundle.min.js"></script>
<link rel="stylesheet" type="text/css" href="/font-awesome/css/font-awesome.min.css">
<script src="/js/anchor.min.js"></script>
<script src="/js/flink.js"></script>
<link rel="canonical" href="https://flink.apache.org/2023/10/24/announcing-the-release-of-apache-flink-1.18/">
<meta charset="UTF-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<meta name="description" content="The Apache Flink PMC is pleased to announce the release of Apache Flink 1.18.0. As usual, we are looking at a packed release with a wide variety of improvements and new features. Overall, 174 people contributed to this release completing 18 FLIPS and 700&#43; issues. Thank you!
Let&rsquo;s dive into the highlights.
Towards a Streaming Lakehouse # Flink SQL Improvements # Introduce Flink JDBC Driver For SQL Gateway # Flink 1.">
<meta name="theme-color" content="#FFFFFF"><meta property="og:title" content="Announcing the Release of Apache Flink 1.18" />
<meta property="og:description" content="The Apache Flink PMC is pleased to announce the release of Apache Flink 1.18.0. As usual, we are looking at a packed release with a wide variety of improvements and new features. Overall, 174 people contributed to this release completing 18 FLIPS and 700&#43; issues. Thank you!
Let&rsquo;s dive into the highlights.
Towards a Streaming Lakehouse # Flink SQL Improvements # Introduce Flink JDBC Driver For SQL Gateway # Flink 1." />
<meta property="og:type" content="article" />
<meta property="og:url" content="https://flink.apache.org/2023/10/24/announcing-the-release-of-apache-flink-1.18/" /><meta property="article:section" content="posts" />
<meta property="article:published_time" content="2023-10-24T22:00:00+00:00" />
<meta property="article:modified_time" content="2023-10-24T22:00:00+00:00" />
<title>Announcing the Release of Apache Flink 1.18 | Apache Flink</title>
<link rel="manifest" href="/manifest.json">
<link rel="icon" href="/favicon.png" type="image/x-icon">
<link rel="stylesheet" href="/book.min.22eceb4d17baa9cdc0f57345edd6f215a40474022dfee39b63befb5fb3c596b5.css" integrity="sha256-IuzrTRe6qc3A9XNF7dbyFaQEdAIt/uObY777X7PFlrU=">
<script defer src="/en.search.min.2698f0d1b683dae4d6cb071668b310a55ebcf1c48d11410a015a51d90105b53e.js" integrity="sha256-Jpjw0baD2uTWywcWaLMQpV688cSNEUEKAVpR2QEFtT4="></script>
<!--
Made with Book Theme
https://github.com/alex-shpak/hugo-book
-->
<meta name="generator" content="Hugo 0.124.1">
<script>
var _paq = window._paq = window._paq || [];
_paq.push(['disableCookies']);
_paq.push(["setDomains", ["*.flink.apache.org","*.nightlies.apache.org/flink"]]);
_paq.push(['trackPageView']);
_paq.push(['enableLinkTracking']);
(function() {
var u="//analytics.apache.org/";
_paq.push(['setTrackerUrl', u+'matomo.php']);
_paq.push(['setSiteId', '1']);
var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0];
g.async=true; g.src=u+'matomo.js'; s.parentNode.insertBefore(g,s);
})();
</script>
</head>
<body dir=ZgotmplZ>
<header>
<nav class="navbar navbar-expand-xl">
<div class="container-fluid">
<a class="navbar-brand" href="/">
<img src="/img/logo/png/100/flink_squirrel_100_color.png" alt="Apache Flink" height="47" width="47" class="d-inline-block align-text-middle">
<span>Apache Flink</span>
</a>
<button class="navbar-toggler" type="button" data-bs-toggle="collapse" data-bs-target="#navbarSupportedContent" aria-controls="navbarSupportedContent" aria-expanded="false" aria-label="Toggle navigation">
<i class="fa fa-bars navbar-toggler-icon"></i>
</button>
<div class="collapse navbar-collapse" id="navbarSupportedContent">
<ul class="navbar-nav">
<li class="nav-item dropdown">
<a class="nav-link dropdown-toggle" href="#" role="button" data-bs-toggle="dropdown" aria-expanded="false">About</a>
<ul class="dropdown-menu">
<li>
<a class="dropdown-item" href="/what-is-flink/flink-architecture/">Architecture</a>
</li>
<li>
<a class="dropdown-item" href="/what-is-flink/flink-applications/">Applications</a>
</li>
<li>
<a class="dropdown-item" href="/what-is-flink/flink-operations/">Operations</a>
</li>
<li>
<a class="dropdown-item" href="/what-is-flink/use-cases/">Use Cases</a>
</li>
<li>
<a class="dropdown-item" href="/what-is-flink/powered-by/">Powered By</a>
</li>
<li>
<a class="dropdown-item" href="/what-is-flink/roadmap/">Roadmap</a>
</li>
<li>
<a class="dropdown-item" href="/what-is-flink/community/">Community & Project Info</a>
</li>
<li>
<a class="dropdown-item" href="/what-is-flink/security/">Security</a>
</li>
<li>
<a class="dropdown-item" href="/what-is-flink/special-thanks/">Special Thanks</a>
</li>
</ul>
</li>
<li class="nav-item dropdown">
<a class="nav-link dropdown-toggle" href="#" role="button" data-bs-toggle="dropdown" aria-expanded="false">Getting Started</a>
<ul class="dropdown-menu">
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-stable/docs/try-flink/local_installation/">With Flink<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-stable/docs/try-flink-kubernetes-operator/quick-start/">With Flink Kubernetes Operator<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-cdc-docs-stable/docs/get-started/introduction/">With Flink CDC<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-ml-docs-stable/docs/try-flink-ml/quick-start/">With Flink ML<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-statefun-docs-stable/getting-started/project-setup.html">With Flink Stateful Functions<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-stable/docs/learn-flink/overview/">Training Course<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
</ul>
</li>
<li class="nav-item dropdown">
<a class="nav-link dropdown-toggle" href="#" role="button" data-bs-toggle="dropdown" aria-expanded="false">Documentation</a>
<ul class="dropdown-menu">
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-stable/">Flink 1.19 (stable)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-docs-master/">Flink Master (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-stable/">Kubernetes Operator 1.8 (latest)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-main">Kubernetes Operator Main (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-cdc-docs-stable">CDC 3.0 (stable)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-cdc-docs-master">CDC Master (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-ml-docs-stable/">ML 2.3 (stable)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-ml-docs-master">ML Master (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-statefun-docs-stable/">Stateful Functions 3.3 (stable)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
<li>
<a class="dropdown-item" href="https://nightlies.apache.org/flink/flink-statefun-docs-master">Stateful Functions Master (snapshot)<i class="link fa fa-external-link title" aria-hidden="true"></i>
</a>
</li>
</ul>
</li>
<li class="nav-item dropdown">
<a class="nav-link dropdown-toggle" href="#" role="button" data-bs-toggle="dropdown" aria-expanded="false">How to Contribute</a>
<ul class="dropdown-menu">
<li>
<a class="dropdown-item" href="/how-to-contribute/overview/">Overview</a>
</li>
<li>
<a class="dropdown-item" href="/how-to-contribute/contribute-code/">Contribute Code</a>
</li>
<li>
<a class="dropdown-item" href="/how-to-contribute/reviewing-prs/">Review Pull Requests</a>
</li>
<li>
<a class="dropdown-item" href="/how-to-contribute/code-style-and-quality-preamble/">Code Style and Quality Guide</a>
</li>
<li>
<a class="dropdown-item" href="/how-to-contribute/contribute-documentation/">Contribute Documentation</a>
</li>
<li>
<a class="dropdown-item" href="/how-to-contribute/documentation-style-guide/">Documentation Style Guide</a>
</li>
<li>
<a class="dropdown-item" href="/how-to-contribute/improve-website/">Contribute to the Website</a>
</li>
<li>
<a class="dropdown-item" href="/how-to-contribute/getting-help/">Getting Help</a>
</li>
</ul>
</li>
<li class="nav-item">
<a class="nav-link" href="/posts/">Flink Blog</a>
</li>
<li class="nav-item">
<a class="nav-link" href="/downloads/">Downloads</a>
</li>
</ul>
<div class="book-search">
<div class="book-search-spinner hidden">
<i class="fa fa-refresh fa-spin"></i>
</div>
<form class="search-bar d-flex" onsubmit="return false;"su>
<input type="text" id="book-search-input" placeholder="Search" aria-label="Search" maxlength="64" data-hotkeys="s/">
<i class="fa fa-search search"></i>
<i class="fa fa-circle-o-notch fa-spin spinner"></i>
</form>
<div class="book-search-spinner hidden"></div>
<ul id="book-search-results"></ul>
</div>
</div>
</div>
</nav>
<div class="navbar-clearfix"></div>
</header>
<main class="flex">
<section class="container book-page">
<article class="markdown">
<h1>
<a href="/2023/10/24/announcing-the-release-of-apache-flink-1.18/">Announcing the Release of Apache Flink 1.18</a>
</h1>
October 24, 2023 -
Jing Ge
<a href="https://twitter.com/jingengineer">(@jingengineer)</a>
Konstantin Knauf
<a href="https://twitter.com/snntrable">(@snntrable)</a>
Sergey Nuyanzin
<a href="https://twitter.com/uckamello">(@uckamello)</a>
Qingsheng Ren
<a href="https://twitter.com/renqstuite">(@renqstuite)</a>
<p><p>The Apache Flink PMC is pleased to announce the release of Apache Flink 1.18.0. As usual, we are looking at a packed
release with a wide variety of improvements and new features. Overall, 174 people contributed to this release completing
18 FLIPS and 700+ issues. Thank you!</p>
<p>Let&rsquo;s dive into the highlights.</p>
<h1 id="towards-a-streaming-lakehouse">
Towards a Streaming Lakehouse
<a class="anchor" href="#towards-a-streaming-lakehouse">#</a>
</h1>
<h2 id="flink-sql-improvements">
Flink SQL Improvements
<a class="anchor" href="#flink-sql-improvements">#</a>
</h2>
<h3 id="introduce-flink-jdbc-driver-for-sql-gateway">
Introduce Flink JDBC Driver For SQL Gateway
<a class="anchor" href="#introduce-flink-jdbc-driver-for-sql-gateway">#</a>
</h3>
<p>Flink 1.18 comes with a JDBC Driver for the Flink SQL Gateway. So, you can now use any SQL Client that supports JDBC to
interact with your tables via Flink SQL. Here is an example using <a href="https://julianhyde.github.io/sqlline/manual.html">SQLLine</a>.</p>
<div class="highlight"><pre tabindex="0" class="chroma"><code class="language-shell" data-lang="shell"><span class="line"><span class="cl">sqlline&gt; !connect jdbc:flink://localhost:8083
</span></span></code></pre></div><div class="highlight"><pre tabindex="0" class="chroma"><code class="language-shell" data-lang="shell"><span class="line"><span class="cl">sqlline version 1.12.0
</span></span><span class="line"><span class="cl">sqlline&gt; !connect jdbc:flink://localhost:8083
</span></span><span class="line"><span class="cl">Enter username <span class="k">for</span> jdbc:flink://localhost:8083:
</span></span><span class="line"><span class="cl">Enter password <span class="k">for</span> jdbc:flink://localhost:8083:
</span></span><span class="line"><span class="cl">0: jdbc:flink://localhost:8083&gt; CREATE TABLE T<span class="o">(</span>
</span></span><span class="line"><span class="cl">. . . . . . . . . . . . . . .<span class="o">)</span>&gt; a INT,
</span></span><span class="line"><span class="cl">. . . . . . . . . . . . . . .<span class="o">)</span>&gt; b VARCHAR<span class="o">(</span>10<span class="o">)</span>
</span></span><span class="line"><span class="cl">. . . . . . . . . . . . . . .<span class="o">)</span>&gt; <span class="o">)</span> WITH <span class="o">(</span>
</span></span><span class="line"><span class="cl">. . . . . . . . . . . . . . .<span class="o">)</span>&gt; <span class="s1">&#39;connector&#39;</span> <span class="o">=</span> <span class="s1">&#39;filesystem&#39;</span>,
</span></span><span class="line"><span class="cl">. . . . . . . . . . . . . . .<span class="o">)</span>&gt; <span class="s1">&#39;path&#39;</span> <span class="o">=</span> <span class="s1">&#39;file:///tmp/T.csv&#39;</span>,
</span></span><span class="line"><span class="cl">. . . . . . . . . . . . . . .<span class="o">)</span>&gt; <span class="s1">&#39;format&#39;</span> <span class="o">=</span> <span class="s1">&#39;csv&#39;</span>
</span></span><span class="line"><span class="cl">. . . . . . . . . . . . . . .<span class="o">)</span>&gt; <span class="o">)</span><span class="p">;</span>
</span></span><span class="line"><span class="cl">No rows affected <span class="o">(</span>0.122 seconds<span class="o">)</span>
</span></span><span class="line"><span class="cl">0: jdbc:flink://localhost:8083&gt; INSERT INTO T VALUES <span class="o">(</span>1, <span class="s1">&#39;Hi&#39;</span><span class="o">)</span>, <span class="o">(</span>2, <span class="s1">&#39;Hello&#39;</span><span class="o">)</span><span class="p">;</span>
</span></span><span class="line"><span class="cl">+----------------------------------+
</span></span><span class="line"><span class="cl"><span class="p">|</span> job id <span class="p">|</span>
</span></span><span class="line"><span class="cl">+----------------------------------+
</span></span><span class="line"><span class="cl"><span class="p">|</span> fbade1ab4450fc57ebd5269fdf60dcfd <span class="p">|</span>
</span></span><span class="line"><span class="cl">+----------------------------------+
</span></span><span class="line"><span class="cl"><span class="m">1</span> row selected <span class="o">(</span>1.282 seconds<span class="o">)</span>
</span></span><span class="line"><span class="cl">0: jdbc:flink://localhost:8083&gt; SELECT * FROM T<span class="p">;</span>
</span></span><span class="line"><span class="cl">+---+-------+
</span></span><span class="line"><span class="cl"><span class="p">|</span> a <span class="p">|</span> b <span class="p">|</span>
</span></span><span class="line"><span class="cl">+---+-------+
</span></span><span class="line"><span class="cl"><span class="p">|</span> <span class="m">1</span> <span class="p">|</span> Hi <span class="p">|</span>
</span></span><span class="line"><span class="cl"><span class="p">|</span> <span class="m">2</span> <span class="p">|</span> Hello <span class="p">|</span>
</span></span><span class="line"><span class="cl">+---+-------+
</span></span><span class="line"><span class="cl"><span class="m">2</span> rows selected <span class="o">(</span>1.955 seconds<span class="o">)</span>
</span></span><span class="line"><span class="cl">0: jdbc:flink://localhost:8083&gt;
</span></span></code></pre></div><p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/jdbcdriver/">Documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-293%3A&#43;Introduce&#43;Flink&#43;Jdbc&#43;Driver&#43;For&#43;Sql&#43;Gateway">FLIP-293: Introduce Flink Jdbc Driver For SQL Gateway</a></li>
</ul>
<h3 id="stored-procedure-support-for-flink-connectors">
Stored Procedure Support for Flink Connectors
<a class="anchor" href="#stored-procedure-support-for-flink-connectors">#</a>
</h3>
<p>Stored procedures have been an indispensable tool in traditional databases,
offering a convenient way to encapsulate complex logic for data manipulation
and administrative tasks. They also offer the potential for enhanced
performance, since they can trigger the handling of data operations directly
within an external database. Other popular data systems like Trino and Iceberg
automate and simplify common maintenance tasks into small sets of procedures,
which greatly reduces users&rsquo; administrative burden.</p>
<p>This new update primarily targets developers of Flink connectors, who can now
predefine custom stored procedures into connectors via the Catalog interface.
The primary benefit to users is that connector-specific tasks that previously
may have required writing custom Flink code can now be replaced with simple
calls that encapsulate, standardize, and potentially optimize the underlying
operations. Users can execute procedures using the familiar <code>CALL</code> syntax, and
discover a connector&rsquo;s available procedures with <code>SHOW PROCEDURES</code>. Stored
procedures within connectors improves the extensibility of Flink&rsquo;s SQL and
Table APIs, and should unlock smoother data access and management for users.</p>
<p>Users can use <code>CALL</code> to directly call built-in stored procedures provided by their catalog. For the built-in
stored procedures in catalog, please refer to the documentation of the corresponding catalog. For example, when using
the Apache Paimon catalog, you can use a stored procedure to trigger compaction for a table.</p>
<div class="highlight"><pre tabindex="0" class="chroma"><code class="language-sql" data-lang="sql"><span class="line"><span class="cl"><span class="k">CREATE</span><span class="w"> </span><span class="k">TABLE</span><span class="w"> </span><span class="o">`</span><span class="n">paimon</span><span class="o">`</span><span class="p">.</span><span class="o">`</span><span class="k">default</span><span class="o">`</span><span class="p">.</span><span class="o">`</span><span class="n">T</span><span class="o">`</span><span class="w"> </span><span class="p">(</span><span class="w">
</span></span></span><span class="line"><span class="cl"><span class="w"></span><span class="n">id</span><span class="w"> </span><span class="nb">BIGINT</span><span class="w"> </span><span class="k">PRIMARY</span><span class="w"> </span><span class="k">KEY</span><span class="w"> </span><span class="k">NOT</span><span class="w"> </span><span class="n">ENFORCED</span><span class="p">,</span><span class="w">
</span></span></span><span class="line"><span class="cl"><span class="w"></span><span class="n">dt</span><span class="w"> </span><span class="n">STRING</span><span class="p">,</span><span class="w"> </span><span class="c1">-- format &#39;yyyy-MM-dd&#39;
</span></span></span><span class="line"><span class="cl"><span class="c1"></span><span class="n">v</span><span class="w"> </span><span class="n">STRING</span><span class="w">
</span></span></span><span class="line"><span class="cl"><span class="w"></span><span class="p">);</span><span class="w">
</span></span></span><span class="line"><span class="cl"><span class="w">
</span></span></span><span class="line"><span class="cl"><span class="w"></span><span class="c1">-- use catalog before call procedures
</span></span></span><span class="line"><span class="cl"><span class="c1"></span><span class="n">USE</span><span class="w"> </span><span class="k">CATALOG</span><span class="w"> </span><span class="o">`</span><span class="n">paimon</span><span class="o">`</span><span class="p">;</span><span class="w">
</span></span></span><span class="line"><span class="cl"><span class="w">
</span></span></span><span class="line"><span class="cl"><span class="w"></span><span class="c1">-- compact the whole table using call statement
</span></span></span><span class="line"><span class="cl"><span class="c1"></span><span class="k">CALL</span><span class="w"> </span><span class="n">sys</span><span class="p">.</span><span class="n">compact</span><span class="p">(</span><span class="s1">&#39;default.T&#39;</span><span class="p">);</span><span class="w">
</span></span></span></code></pre></div><p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/procedures/">Documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-311%3A&#43;Support&#43;Call&#43;Stored&#43;Procedure">FLIP-311: Support Call Stored Procedure</a></li>
</ul>
<h3 id="extended-ddl-support">
Extended DDL Support
<a class="anchor" href="#extended-ddl-support">#</a>
</h3>
<p>From this release onwards, Flink supports</p>
<ul>
<li><code>REPLACE TABLE AS SELECT</code></li>
<li><code>CREATE OR REPLACE TABLE AS SELECT</code></li>
</ul>
<p>and both these commands and previously supported <code>CREATE TABLE AS</code> can now support atomicity provided the underlying
connector also supports this.</p>
<p>Moreover, Apache Flink now supports TRUNCATE TABLE in batch execution mode. Same as before, the underlying connector needs
to implement and provide this capability</p>
<p>And, finally, we have also implemented support for adding, dropping and listing partitions via</p>
<ul>
<li><code>ALTER TABLE ADD PARTITION</code></li>
<li><code>ALTER TABLE DROP PARTITION</code></li>
<li><code>SHOW PARTITIONS</code></li>
</ul>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/sql/truncate/">Documentation on TRUNCATE</a></li>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/sql/create/#create-or-replace-table">Documentation on CREATE OR REPLACE</a></li>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/sql/alter/#alter-table">Documentation on ALTER TABLE</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-302%3A&#43;Support&#43;TRUNCATE&#43;TABLE&#43;statement&#43;in&#43;batch&#43;mode">FLIP-302: Support TRUNCATE TABLE statement in batch mode</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-303%3A&#43;Support&#43;REPLACE&#43;TABLE&#43;AS&#43;SELECT&#43;statement">FLIP-303: Support REPLACE TABLE AS SELECT statement</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-305%3A&#43;Support&#43;atomic&#43;for&#43;CREATE&#43;TABLE&#43;AS&#43;SELECT%28CTAS%29&#43;statement">FLIP-305: Support atomic for CREATE TABLE AS SELECT(CTAS) statement</a></li>
</ul>
<h3 id="time-traveling">
Time Traveling
<a class="anchor" href="#time-traveling">#</a>
</h3>
<p>Flink supports the time travel SQL syntax for querying historical versions of data that allows users to specify a point
in time and retrieve the data and schema of a table as it appeared at that time. With time travel, users can easily
analyze and compare historical versions of data.</p>
<p>For example, a user can query a table at a specified point in time with the following statement;</p>
<div class="highlight"><pre tabindex="0" class="chroma"><code class="language-sql" data-lang="sql"><span class="line"><span class="cl"><span class="c1">-- Query the table `tb` for data on November 11, 2022
</span></span></span><span class="line"><span class="cl"><span class="c1"></span><span class="k">SELECT</span><span class="w"> </span><span class="o">*</span><span class="w"> </span><span class="k">FROM</span><span class="w"> </span><span class="n">tb</span><span class="w"> </span><span class="k">FOR</span><span class="w"> </span><span class="n">SYSTEM_TIME</span><span class="w"> </span><span class="k">AS</span><span class="w"> </span><span class="k">OF</span><span class="w"> </span><span class="k">TIMESTAMP</span><span class="w"> </span><span class="s1">&#39;2022-11-11 00:00:00&#39;</span><span class="p">;</span><span class="w">
</span></span></span></code></pre></div><p><strong>More information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/sql/queries/time-travel/">Documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-308%3A&#43;Support&#43;Time&#43;Travel">FLIP-308: Support Time Travel</a></li>
</ul>
<h2 id="streaming-execution-improvements">
Streaming Execution Improvements
<a class="anchor" href="#streaming-execution-improvements">#</a>
</h2>
<h3 id="support-operator-level-state-ttl-in-table-api--sql">
Support Operator-Level State TTL in Table API &amp; SQL
<a class="anchor" href="#support-operator-level-state-ttl-in-table-api--sql">#</a>
</h3>
<p>Starting from Flink 1.18, Table API and SQL users can set state time-to-live (TTL) individually for stateful operators.
This means that for scenarios like stream regular joins, users can now set different TTLs for the left and right
streams. In previous versions, state expiration could only be controlled at the pipeline level using the configuration
<code>table.exec.state.ttl</code>. With the introduction of operator-level state retention, users can now optimize resource
usage according to their specific requirements.</p>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/concepts/overview/#configure-operator-level-state-ttl">Documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-292%3A&#43;Enhance&#43;COMPILED&#43;PLAN&#43;to&#43;support&#43;operator-level&#43;state&#43;TTL&#43;configuration">FLIP-292: Enhance COMPILED PLAN to support operator-level state TTL configuration</a></li>
</ul>
<h3 id="watermark-alignment-and-idleness-detection-in-sql">
Watermark Alignment and Idleness Detection in SQL
<a class="anchor" href="#watermark-alignment-and-idleness-detection-in-sql">#</a>
</h3>
<p>You can now configure <a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/event-time/generating_watermarks/#watermark-alignment">watermark alignment</a>
and <a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/event-time/generating_watermarks/#dealing-with-idle-sources">source idleness timeouts</a>
in <a href="https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/concepts/time_attributes/#advanced-watermark-features">pure SQL via hints</a>. Previously, these features were only available in the DataStream API.</p>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/table/sql/queries/hints/">Documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-296%3A&#43;Extend&#43;watermark-related&#43;features&#43;for&#43;SQL">FLIP-296: Extend watermark-related features for SQL</a></li>
</ul>
<h2 id="batch-execution-improvements">
Batch Execution Improvements
<a class="anchor" href="#batch-execution-improvements">#</a>
</h2>
<h3 id="hybrid-shuffle-supports-remote-storage">
Hybrid Shuffle supports Remote Storage
<a class="anchor" href="#hybrid-shuffle-supports-remote-storage">#</a>
</h3>
<p>Hybrid Shuffle supports storing the shuffle data in remote storage. The remote storage path can be configured with the
option <code>taskmanager.network.hybrid-shuffle.remote.path</code>. Hybrid Shuffle uses less network memory than before by
decoupling the memory usage from the number of parallelisms, improving the stability and ease of use.</p>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/ops/batch/batch_shuffle/#hybrid-shuffle">Documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-301%3A&#43;Hybrid&#43;Shuffle&#43;supports&#43;Remote&#43;Storage">FLIP-301: Hybrid Shuffle supports Remote Storage</a></li>
</ul>
<h3 id="performance-improvements--tpc-ds-benchmark">
Performance Improvements &amp; TPC-DS Benchmark
<a class="anchor" href="#performance-improvements--tpc-ds-benchmark">#</a>
</h3>
<p>In previous releases, the community worked extensively to improve Flink&rsquo;s batch processing performance, which has led to
significant improvements. In this release cycle, community contributors continued to put significant effort into
further improving Flink&rsquo;s batch performance.</p>
<h4 id="runtime-filter-for-flink-sql">
Runtime Filter for Flink SQL
<a class="anchor" href="#runtime-filter-for-flink-sql">#</a>
</h4>
<p>Runtime filter is a common method for optimizing Join performance. It is designed to dynamically generate filter
conditions for certain Join queries at runtime to reduce the amount of scanned or shuffled data, avoid unnecessary I/O
and network transmission, and speed up the query. We introduced runtime filters in Flink 1.18, and verified its
effectiveness through the TPC-DS benchmark, and observed up to 3x speedup for some queries by enabling this feature.</p>
<h4 id="operator-fusion-codegen-for-flink-sql">
Operator Fusion Codegen for Flink SQL
<a class="anchor" href="#operator-fusion-codegen-for-flink-sql">#</a>
</h4>
<p>Operator Fusion Codegen improves the execution performance of a query by fusing an operator DAG into a single optimized
operator that eliminates virtual function calls, leverages CPU registers for intermediate data and reduces the
instruction cache miss. As a general technical optimization, we verified its effectiveness through TPC-DS, and
only some batch operators (Calc, HashAgg, and HashJoin) completed fusion codegen support in version 1.18, getting
significant performance gains on some query.</p>
<p>Note that both features are disabled by default in Flink 1.18 and the Community is looking for feedback by users before
enabling them by default. They can be enabled by using <code>table.optimizer.runtime-filter.enabled</code> and
<code>table.exec.operator-fusion-codegen.enabled</code> respectively.</p>
<p>Since Flink 1.16, the Apache Flink Community has been continuously tracking the performance of its batch engine via the
TPC-DS benchmarking framework. After significant improvements in Flink 1.17 (dynamic join-reordering,
dynamic local aggregations), the two improvements described in the previous sections (operator fusion, runtime filters)
lead to 14% performance improvement compared to Flink 1.17, a 54% performance improvement compared to Flink 1.16 on a
10T dataset for partitioned tables.</p>
<div style="text-align: center;">
<img src="/img/blog/2023-10-24-release-1.18.0/tpc-ds-benchmark.png" style="width:90%;margin:15px">
</div>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-324%3A&#43;Introduce&#43;Runtime&#43;Filter&#43;for&#43;Flink&#43;Batch&#43;Jobs">FLIP-324: Introduce Runtime Filter for Flink Batch Jobs</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-315&#43;Support&#43;Operator&#43;Fusion&#43;Codegen&#43;for&#43;Flink&#43;SQL">FLIP-315: Support Operator Fusion Codegen for Flink SQL</a></li>
<li><a href="https://github.com/ververica/flink-sql-benchmark">Benchmarking repository</a></li>
</ul>
<h1 id="towards-cloud-native-elasticity">
Towards Cloud-Native Elasticity
<a class="anchor" href="#towards-cloud-native-elasticity">#</a>
</h1>
<p>Elasticity describes the ability of a system to adapt to workload changes in a non-disruptive, ideally automatic manner.
It is a defining characteristic of cloud-native systems and for long-running streaming workloads it is particularly
important. As such, elasticity improvements are an area of continuous investment in the Apache Flink community.
Recent initiatives include the Kubernetes
<a href="https://nightlies.apache.org/flink/flink-kubernetes-operator-docs-release-1.6/docs/custom-resource/autoscaler/">Autoscaler</a>,
numerous improvements to rescaling performance and last but not least
the <a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/elastic_scaling/#adaptive-scheduler">Adaptive Scheduler</a>.</p>
<p>The Adaptive Scheduler was first introduced in Flink 1.15 and constitutes a centerpiece of a fully-elastic
Apache Flink deployment. At its core, it allows jobs to change their resource requirements and parallelism during
runtime. In addition, it also adapts to the available resources in the cluster by only rescaling once the cluster can
satisfy the minimum required resources of the job.</p>
<p>Until Flink 1.18, the adaptive scheduler was primarily used in
<a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/elastic_scaling/#reactive-mode">Reactive Mode</a>,
which meant that a single job by design would always use all the available resources in the cluster.
Please see <a href="https://flink.apache.org/2021/05/06/scaling-flink-automatically-with-reactive-mode/">this blog post</a> on
how to autoscale Flink Jobs in Reactive Mode using a Horizontal Pod Autoscaler on Kubernetes.</p>
<p>With Flink 1.18 the adaptive scheduler becomes much more powerful and more widely applicable and is on a trajectory to
becoming the default scheduler for streaming workloads on Apache Flink.</p>
<h2 id="dynamic-fine-grained-rescaling-via-rest-api">
Dynamic Fine-Grained Rescaling via REST API
<a class="anchor" href="#dynamic-fine-grained-rescaling-via-rest-api">#</a>
</h2>
<p>Despite the underlying capabilities of the Adaptive Scheduler, the ability to change the resource requirements of a
Job during runtime has not yet been exposed to the end user directly. This changes in Flink 1.18. You can now change
the parallelism of any individual task of your job via the Flink Web UI and REST API while the job is running.</p>
<div style="position: relative; padding-bottom: 56.25%; height: 0; overflow: hidden;">
<iframe src="https://www.youtube.com/embed/B1NVDTazsZY" style="position: absolute; top: 0; left: 0; width: 100%; height: 100%; border:0;" allowfullscreen title="YouTube Video"></iframe>
</div>
<p>Under the hood, Apache Flink performs a regular rescaling operation as soon as the required resources for the new
parallelism have been acquired. The rescaling operation is not based on a Savepoint, but on an ordinary, periodic
checkpoint, which means it does not introduce any additional snapshot. As you can see in the video above, the rescaling
operation already happens nearly instantaneously and with a very short downtime for jobs with small state size.</p>
<p>In conjunction with the
<a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/ops/monitoring/back_pressure/">backpressure monitor</a>
of the Apache Flink Web UI, it is now easier than ever to find and maintain an efficient, backpressure-free parallelism
for each of the tasks:</p>
<ul>
<li>If a task is very busy (red), you increase the parallelism.</li>
<li>If a task is mostly idle (blue), you decrease the parallelism.</li>
</ul>
<div style="text-align: center;">
<img src="/img/blog/2023-10-24-release-1.18.0/backpressure_monitor.png" style="width:90%;margin:15px">
</div>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/elastic_scaling/#adaptive-scheduler">Documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-291%3A&#43;Externalized&#43;Declarative&#43;Resource&#43;Management">FLIP-291: Externalized Declarative Resource Management</a></li>
</ul>
<h2 id="faster-rescaling-with-rocksdb">
Faster Rescaling with RocksDB
<a class="anchor" href="#faster-rescaling-with-rocksdb">#</a>
</h2>
<p>The rescaling times when using RocksDB Statebackend with incremental checkpoints have been improved about 30% in the 99th quantile.</p>
<p>We increased the potential for parallel download from just downloading state handles in parallel to downloading individual files in parallel.</p>
<p>Furthermore, we deactivated write-ahead-logging for batch-inserting into the temporary RocksDB instances we use for rescaling.</p>
<div style="text-align: center;">
<img src="/img/blog/2023-10-24-release-1.18.0/rescaling_performance.png" style="width:90%;margin:15px">
</div>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://issues.apache.org/jira/browse/FLINK-32326">FLINK-32326</a></li>
<li><a href="https://issues.apache.org/jira/browse/FLINK-32345">FLINK-32345</a></li>
</ul>
<h1 id="support-for-java-17">
Support for Java 17
<a class="anchor" href="#support-for-java-17">#</a>
</h1>
<p>Java 17 was released in 2021 and is the latest long-term support (LTS) release of Java with an end-of-life in 2029.
So, it was about time that Apache Flink added support for it. What does this mean concretely? As of Flink 1.18, you can
now run Apache Flink on Java 17 and the <a href="https://hub.docker.com/_/flink">official Docker repository</a> includes an image
based on Java 17.</p>
<pre><code>docker pull flink:1.18.0-java17
</code></pre>
<p>If your cluster runs on Java 17, this of course, also allows you to use Java 17 features in your user programs and to
compile it to a target version of Java 17.</p>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/java_compatibility/">Documentation</a></li>
<li><a href="https://issues.apache.org/jira/browse/FLINK-15736">FLINK-15736</a></li>
</ul>
<h1 id="others-improvements">
Others Improvements
<a class="anchor" href="#others-improvements">#</a>
</h1>
<h2 id="production-ready-watermark-alignment">
Production-Ready Watermark Alignment
<a class="anchor" href="#production-ready-watermark-alignment">#</a>
</h2>
<p>Supported as “Beta” since Flink 1.16 and Flink 1.17,
<a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/event-time/generating_watermarks/#watermark-alignment">watermark alignment</a>
has been thoroughly tested at scale in the real world. Over that time the community has collected and addressed
bugs and performance issues as they were discovered. With the resolution of these issues, we are now happy to recommend
watermark alignment for general use.</p>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://issues.apache.org/jira/browse/FLINK-32548">FLINK-32548</a></li>
</ul>
<h2 id="pluggable-failure-handling">
Pluggable Failure Handling
<a class="anchor" href="#pluggable-failure-handling">#</a>
</h2>
<p>Apache Flink serves as the foundation for numerous stream processing platforms at companies like Apple, Netflix or Uber. It is also the basis for various commercial stream processing services. Therefore, its ability to easily integrate into the wider ecosystem of these internal as well as vendor platforms becomes increasingly important. The catalog modification listener and pluggable failure handlers fall into this category of improvements.</p>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/deployment/advanced/failure_enrichers/">Documentation</a></li>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-304%3A&#43;Pluggable&#43;Failure&#43;Enrichers">FLIP-304: Pluggable Failure Enrichers</a></li>
</ul>
<h2 id="sql-client-quality-of-life-improvements">
SQL Client Quality of Life Improvements
<a class="anchor" href="#sql-client-quality-of-life-improvements">#</a>
</h2>
<p>In 1.18 the SQL Client received a collection of usability improvements:</p>
<ul>
<li>The SQL Client is now more colorful with the ability to enable SQL syntax highlighting and switching among 7 different
color schemes</li>
<li>It is now easier to edit and navigate through very large queries.</li>
<li>It is now possible to turn line numbers off and on.</li>
</ul>
<p>More Information</p>
<ul>
<li><a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-189%3A&#43;SQL&#43;Client&#43;Usability&#43;Improvements">FLIP-189: SQL Client Usability Improvements</a></li>
</ul>
<h2 id="apache-pekko-instead-of-akka">
Apache Pekko instead of Akka
<a class="anchor" href="#apache-pekko-instead-of-akka">#</a>
</h2>
<p>A year ago, <a href="https://flink.apache.org/2022/09/08/regarding-akkas-licensing-change/">Lightbend announced</a> changing the
license of future versions of Akka (2.7+) from Apache 2.0 to BSL. It was also announced that Akka 2.6, the version that
Apache Flink uses, would receive security updates and critical bug fixes until September of 2023. As September 2023 was
approaching, we decided to switch from Akka to <a href="https://pekko.apache.org/">Apache Pekko</a> (incubating).
Apache Pekko (incubating) is a fork of Akka 2.6.x, prior to the Akka project’s adoption of the Business Source License.
Pekko recently released Apache Pekko 1.0.1-incubating, which enabled us to already use it in Flink 1.18 - just in time.
While our mid-term plan is to drop the dependency on Akka or Pekko altogether
(see <a href="https://issues.apache.org/jira/browse/FLINK-29281">FLINK-29281</a>), the switch to Pekko presents a good short-term
solution and ensures that the Apache Pekko and Apache Flink Community can address critical bug fixes and security
vulnerabilities throughout our software supply chain.</p>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://issues.apache.org/jira/browse/FLINK-32468">FLINK-32468</a></li>
</ul>
<h2 id="calcite-upgrades">
Calcite Upgrade(s)
<a class="anchor" href="#calcite-upgrades">#</a>
</h2>
<p>In Apache Flink 1.18, Apache Calcite was gradually upgraded from 1.29 to 1.32. The immediate benefit of these upgrades
are bug fixes, a smarter optimizer and performance improvements. On a parser level, it now allows joins to be
grouped into trees using parentheses (mentioned in SQL-92) e.g. <code>SELECT * FROM a JOIN (b JOIN c ON b.x = c.x) ON a.y = c.y</code>
also see <a href="https://issues.apache.org/jira/browse/CALCITE-35">CALCITE-35</a>. In addition, the upgrade to Calcite 1.31+ has
unblocked the support of Session Windows via Table-Valued Functions (see
<a href="https://issues.apache.org/jira/browse/CALCITE-4865">CALCITE-4865</a>,
<a href="https://issues.apache.org/jira/browse/FLINK-24024">FLINK-24024</a>) and as a corollary the deprecation of the legacy
group window aggregations.
Due to <a href="https://issues.apache.org/jira/browse/CALCITE-4861">CALCITE-4861</a> Flink&rsquo;s casting behavior has slightly changed.
Some corner cases might behave differently now: For example, casting from FLOAT/DOUBLE 9234567891.12 to INT/BIGINT has
now Java behavior for overflows.</p>
<p><strong>More Information</strong></p>
<ul>
<li><a href="https://issues.apache.org/jira/browse/FLINK-27998">FLINK-27998</a></li>
<li><a href="https://issues.apache.org/jira/browse/FLINK-28744">FLINK-28744</a></li>
<li><a href="https://issues.apache.org/jira/browse/FLINK-29319">FLINK-29319</a></li>
</ul>
<h1 id="important-deprecations">
Important Deprecations
<a class="anchor" href="#important-deprecations">#</a>
</h1>
<p>In preparation for the release of Flink 2.0 next year, the community has decided to officially deprecate multiple APIs
that were approaching end of life for a while.</p>
<ul>
<li><a href="https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/functions/source/SourceFunction.java"><code>SourceFunction</code></a>
is now officially deprecated and will be dropped in Flink 2.0. If you are still using a connector that is built on
top of <code>SourceFunction</code> please migrate it to
<a href="https://github.com/apache/flink/blob/master/flink-core/src/main/java/org/apache/flink/api/connector/source/Source.java"><code>Source</code></a>.
<code>SinkFunction</code> is not officially deprecated, but it is also approaching end-of-life and will be superseded by
<code>SinkV2</code>.</li>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.17/docs/dev/datastream/fault-tolerance/queryable_state/">Queryable State</a>
is now officially deprecated and will be dropped in Flink 2.0.</li>
<li><a href="https://nightlies.apache.org/flink/flink-docs-release-1.17/docs/dev/dataset/overview/">The DataSet API</a> is now
officially deprecated. Users are recommended to migrate to the DataStream API with
<a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/docs/dev/datastream/execution_mode/">execution mode</a> <code>BATCH</code>.</li>
</ul>
<h1 id="upgrade-notes">
Upgrade Notes
<a class="anchor" href="#upgrade-notes">#</a>
</h1>
<p>The Flink community tries to ensure that upgrades are as seamless as possible.
However, certain changes may require users to make adjustments to certain parts
of the program when upgrading to version 1.18. Please refer to the
<a href="https://nightlies.apache.org/flink/flink-docs-release-1.18/release-notes/flink-1.18/">release notes</a>
for a comprehensive list of adjustments to make and issues to check during the
upgrading process.</p>
<h1 id="list-of-contributors">
List of Contributors
<a class="anchor" href="#list-of-contributors">#</a>
</h1>
<p>The Apache Flink community would like to express gratitude to all the
contributors who made this release possible:</p>
<p>Aitozi,
Akinfolami Akin-Alamu,
Alain Brown,
Aleksandr Pilipenko,
Alexander Fedulov,
Anton Kalashnikov,
Archit Goyal,
Bangui Dunn,
Benchao Li,
BoYiZhang,
Chesnay Schepler,
Chris Nauroth,
Colten Pilgreen,
Danny Cranmer,
David Christle,
David Moravek,
Dawid Wysakowicz,
Deepyaman Datta,
Dian Fu,
Dian Qi,
Dong Lin,
Eric Xiao,
Etienne Chauchot,
Feng Jin,
Ferenc Csaky,
Fruzsina Nagy,
Gabor Somogyi,
Gunnar Morling,
Gyula Fora,
HaiYang Chen,
Hang Ruan,
Hangxiang Yu,
Hanyu Zheng,
Hong Liang Teoh,
Hongshun Wang,
Huston,
Jacky Lau,
James Hughes,
Jane Chan,
Jark Wu,
Jayadeep Jayaraman,
Jia Liu,
JiangXin,
Joao Boto,
Junrui Lee,
Juntao Hu,
K.I. (Dennis) Jung,
Kaiqi Dong,
L,
Leomax_Sun,
Leonard Xu,
Licho,
Lijie Wang,
Liu Jiangang,
Lyn Zhang,
Maomao Min,
Martijn Visser,
Marton Balassi,
Mason Chen,
Matthew de Detrich,
Matthias Pohl,
Min,
Mingliang Liu,
Mohsen Rezaei,
Mrart,
Mulavar,
Nicholas Jiang,
Nicolas Fraison,
Noah,
Panagiotis Garefalakis,
Patrick Lucas,
Paul Lin,
Peter Vary,
Piotr Nowojski,
Qingsheng Ren,
Ran Tao,
Rich Bowen,
Robert Metzger,
Roc Marshal,
Roman Khachatryan,
Ron,
Rui Fan,
Ryan Skraba,
Samrat002,
Sergey Nuyanzin,
Sergio Morales,
Shammon FY,
ShammonFY,
Shengkai,
Shuiqiang Chen,
Stefan Richter,
Tartarus0zm,
Timo Walther,
Tzu-Li (Gordon) Tai,
Venkata krishnan Sowrirajan,
Wang FeiFan,
Weihua Hu,
Weijie Guo,
Wencong Liu,
Xiaogang Zhou,
Xintong Song,
XuShuai,
Yanfei Lei,
Yu Chen,
Yubin Li,
Yun Gao,
Yun Tang,
Yuxin Tan,
Zakelly,
Zhanghao Chen,
ZhengYiWeng,
Zhu Zhu,
archzi,
baiwuchang,
cailiuyang,
chenyuzhi,
darenwkt,
dongwoo kim,
eason.qin,
felixzh,
fengli,
frankeshi,
fredia,
godfrey he,
haishui,
hehuiyuan,
huangxingbo,
jiangxin,
jiaoqingbo,
jinfeng,
jingge,
kevin.cyj,
kristoffSC,
leixin,
leiyanfei,
liming.1018,
lincoln lee,
lincoln.lil,
liujiangang,
liuyongvs,
luoyuxia,
maigeiye,
mas-chen,
novakov-alexey,
oleksandr.nitavskyi,
pegasas,
sammieliu,
shammon,
shammon FY,
shuiqiangchen,
slfan1989,
sunxia,
tison,
tsreaper,
wangfeifan,
wangkang,
whjshj,
wuqqq,
xiangyu0xf,
xincheng.ljr,
xmzhou,
xuyu,
xzw,
yuanweining,
yuchengxin,
yunfengzhou-hub,
yunhong,
yuxia Luo,
yuxiqian,
zekai-li,
zhangmang,
zhengyunhong.zyh,
zzzzzzzs,
沈嘉琦</p>
</p>
</article>
<div class="edit-this-page">
<p>
<a href="https://cwiki.apache.org/confluence/display/FLINK/Flink+Translation+Specifications">Want to contribute translation?</a>
</p>
<p>
<a href="//github.com/apache/flink-web/edit/asf-site/docs/content/posts/2023-10-24-release-1.18.0.md">
Edit This Page<i class="fa fa-edit fa-fw"></i>
</a>
</p>
</div>
</section>
<aside class="book-toc">
<nav id="TableOfContents"><h3>On This Page <a href="javascript:void(0)" class="toc" onclick="collapseToc()"><i class="fa fa-times" aria-hidden="true"></i></a></h3>
<ul>
<li><a href="#towards-a-streaming-lakehouse">Towards a Streaming Lakehouse</a>
<ul>
<li><a href="#flink-sql-improvements">Flink SQL Improvements</a>
<ul>
<li><a href="#introduce-flink-jdbc-driver-for-sql-gateway">Introduce Flink JDBC Driver For SQL Gateway</a></li>
<li><a href="#stored-procedure-support-for-flink-connectors">Stored Procedure Support for Flink Connectors</a></li>
<li><a href="#extended-ddl-support">Extended DDL Support</a></li>
<li><a href="#time-traveling">Time Traveling</a></li>
</ul>
</li>
<li><a href="#streaming-execution-improvements">Streaming Execution Improvements</a>
<ul>
<li><a href="#support-operator-level-state-ttl-in-table-api--sql">Support Operator-Level State TTL in Table API &amp; SQL</a></li>
<li><a href="#watermark-alignment-and-idleness-detection-in-sql">Watermark Alignment and Idleness Detection in SQL</a></li>
</ul>
</li>
<li><a href="#batch-execution-improvements">Batch Execution Improvements</a>
<ul>
<li><a href="#hybrid-shuffle-supports-remote-storage">Hybrid Shuffle supports Remote Storage</a></li>
<li><a href="#performance-improvements--tpc-ds-benchmark">Performance Improvements &amp; TPC-DS Benchmark</a></li>
</ul>
</li>
</ul>
</li>
<li><a href="#towards-cloud-native-elasticity">Towards Cloud-Native Elasticity</a>
<ul>
<li><a href="#dynamic-fine-grained-rescaling-via-rest-api">Dynamic Fine-Grained Rescaling via REST API</a></li>
<li><a href="#faster-rescaling-with-rocksdb">Faster Rescaling with RocksDB</a></li>
</ul>
</li>
<li><a href="#support-for-java-17">Support for Java 17</a></li>
<li><a href="#others-improvements">Others Improvements</a>
<ul>
<li><a href="#production-ready-watermark-alignment">Production-Ready Watermark Alignment</a></li>
<li><a href="#pluggable-failure-handling">Pluggable Failure Handling</a></li>
<li><a href="#sql-client-quality-of-life-improvements">SQL Client Quality of Life Improvements</a></li>
<li><a href="#apache-pekko-instead-of-akka">Apache Pekko instead of Akka</a></li>
<li><a href="#calcite-upgrades">Calcite Upgrade(s)</a></li>
</ul>
</li>
<li><a href="#important-deprecations">Important Deprecations</a></li>
<li><a href="#upgrade-notes">Upgrade Notes</a></li>
<li><a href="#list-of-contributors">List of Contributors</a></li>
</ul>
</nav>
</aside>
<aside class="expand-toc hidden">
<a class="toc" onclick="expandToc()" href="javascript:void(0)">
<i class="fa fa-bars" aria-hidden="true"></i>
</a>
</aside>
</main>
<footer>
<div class="separator"></div>
<div class="panels">
<div class="wrapper">
<div class="panel">
<ul>
<li>
<a href="https://flink-packages.org/">flink-packages.org</a>
</li>
<li>
<a href="https://www.apache.org/">Apache Software Foundation</a>
</li>
<li>
<a href="https://www.apache.org/licenses/">License</a>
</li>
<li>
<a href="/zh/">
<i class="fa fa-globe" aria-hidden="true"></i>&nbsp;中文版
</a>
</li>
</ul>
</div>
<div class="panel">
<ul>
<li>
<a href="/what-is-flink/security">Security</a-->
</li>
<li>
<a href="https://www.apache.org/foundation/sponsorship.html">Donate</a>
</li>
<li>
<a href="https://www.apache.org/foundation/thanks.html">Thanks</a>
</li>
</ul>
</div>
<div class="panel icons">
<div>
<a href="/posts">
<div class="icon flink-blog-icon"></div>
<span>Flink blog</span>
</a>
</div>
<div>
<a href="https://github.com/apache/flink">
<div class="icon flink-github-icon"></div>
<span>Github</span>
</a>
</div>
<div>
<a href="https://twitter.com/apacheflink">
<div class="icon flink-twitter-icon"></div>
<span>Twitter</span>
</a>
</div>
</div>
</div>
</div>
<hr/>
<div class="container disclaimer">
<p>The contents of this website are © 2024 Apache Software Foundation under the terms of the Apache License v2. Apache Flink, Flink, and the Flink logo are either registered trademarks or trademarks of The Apache Software Foundation in the United States and other countries.</p>
</div>
</footer>
</body>
</html>