blob: dc8bafc8fcf168ac9cdf894ef14981f04e489642 [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/2019/08/22/apache-flink-1.9.0-release-announcement/">
<meta charset="UTF-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<meta name="description" content="The Apache Flink community is proud to announce the release of Apache Flink 1.9.0.
The Apache Flink project&rsquo;s goal is to develop a stream processing system to unify and power many forms of real-time and offline data processing applications as well as event-driven applications. In this release, we have made a huge step forward in that effort, by integrating Flink’s stream and batch processing capabilities under a single, unified runtime.">
<meta name="theme-color" content="#FFFFFF"><meta property="og:title" content="Apache Flink 1.9.0 Release Announcement" />
<meta property="og:description" content="The Apache Flink community is proud to announce the release of Apache Flink 1.9.0.
The Apache Flink project&rsquo;s goal is to develop a stream processing system to unify and power many forms of real-time and offline data processing applications as well as event-driven applications. In this release, we have made a huge step forward in that effort, by integrating Flink’s stream and batch processing capabilities under a single, unified runtime." />
<meta property="og:type" content="article" />
<meta property="og:url" content="https://flink.apache.org/2019/08/22/apache-flink-1.9.0-release-announcement/" /><meta property="article:section" content="posts" />
<meta property="article:published_time" content="2019-08-22T02:30:00+00:00" />
<meta property="article:modified_time" content="2019-08-22T02:30:00+00:00" />
<title>Apache Flink 1.9.0 Release Announcement | 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="/2019/08/22/apache-flink-1.9.0-release-announcement/">Apache Flink 1.9.0 Release Announcement</a>
</h1>
August 22, 2019 -
<p><p>The Apache Flink community is proud to announce the release of Apache Flink
1.9.0.</p>
<p>The Apache Flink project&rsquo;s goal is to develop a stream processing system to
unify and power many forms of real-time and offline data processing
applications as well as event-driven applications. In this release, we have
made a huge step forward in that effort, by integrating Flink’s stream and
batch processing capabilities under a single, unified runtime.</p>
<p>Significant features on this path are batch-style recovery for batch jobs and
a preview of the new Blink-based query engine for Table API and SQL queries.
We are also excited to announce the availability of the State Processor API,
which is one of the most frequently requested features and enables users to
read and write savepoints with Flink DataSet jobs. Finally, Flink 1.9 includes
a reworked WebUI and previews of Flink’s new Python Table API and its
integration with the Apache Hive ecosystem.</p>
<p>This blog post describes all major new features and improvements, important
changes to be aware of and what to expect moving forward. For more details,
check the <a href="https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522&amp;version=12344601">complete release
changelog</a>.</p>
<p>The binary distribution and source artifacts for this release are now
available via the <a href="https://flink.apache.org/downloads.html">Downloads</a> page of
the Flink project, along with the updated
<a href="//nightlies.apache.org/flink/flink-docs-release-1.9/">documentation</a>.
Flink 1.9 is API-compatible with previous 1.x releases for APIs annotated with
the <code>@Public</code> annotation.</p>
<p>Please feel encouraged to download the release and share your thoughts with
the community through the Flink <a href="https://flink.apache.org/community.html#mailing-lists">mailing
lists</a> or
<a href="https://issues.apache.org/jira/projects/FLINK/summary">JIRA</a>. As always,
feedback is very much appreciated!</p>
<h2 id="new-features-and-improvements">
New Features and Improvements
<a class="anchor" href="#new-features-and-improvements">#</a>
</h2>
<h3 id="fine-grained-batch-recovery-flip-1">
Fine-grained Batch Recovery (FLIP-1)
<a class="anchor" href="#fine-grained-batch-recovery-flip-1">#</a>
</h3>
<p>The time to recover a batch (DataSet, Table API and SQL) job from a task
failure was significantly reduced. Until Flink 1.9, task failures in batch
jobs were recovered by canceling all tasks and restarting the whole job, i.e,
the job was started from scratch and all progress was voided. With this
release, Flink can be configured to limit the recovery to only those tasks
that are in the same <strong>failover region</strong>. A failover region is the set of
tasks that are connected via pipelined data exchanges. Hence, the
batch-shuffle connections of a job define the boundaries of its failover
regions. More details are available in
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-1&#43;%3A&#43;Fine&#43;Grained&#43;Recovery&#43;from&#43;Task&#43;Failures">FLIP-1</a>.
<img src="/img/blog/release-19-flip1.png" alt="alt_text" title="Fine-grained Batch
Recovery"/></p>
<p>To use this new failover strategy, you need to do the following
settings:</p>
<ul>
<li>Make sure you have the entry <code>jobmanager.execution.failover-strategy: region</code> in your <code>flink-conf.yaml</code>.</li>
</ul>
<p><strong>Note:</strong> The configuration of the 1.9 distribution has that entry by default,
but when reusing a configuration file from previous setups, you have to add
it manually.</p>
<p>Moreover, you need to set the <code>ExecutionMode</code> of batch jobs in the
<code>ExecutionConfig</code> to <code>BATCH</code> to configure that data shuffles are not pipelined
and jobs have more than one failover region.</p>
<p>The &ldquo;Region&rdquo; failover strategy also improves the recovery of “embarrassingly
parallel” streaming jobs, i.e., jobs without any shuffle like keyBy() or
rebalance. When such a job is recovered, only the tasks of the affected
pipeline (failover region) are restarted. For all other streaming jobs, the
recovery behavior is the same as in prior Flink versions.</p>
<h3 id="state-processor-api-flip-43">
State Processor API (FLIP-43)
<a class="anchor" href="#state-processor-api-flip-43">#</a>
</h3>
<p>Up to Flink 1.9, accessing the state of a job from the outside was limited to
the (still) experimental <a href="//nightlies.apache.org/flink/flink-docs-stable/dev/stream/state/queryable_state.html">Queryable
State</a>.
This release introduces a new, powerful library to read, write and modify
state snapshots using the batch DataSet API. In practice, this means:</p>
<ul>
<li>Flink job state can be bootstrapped by reading data from external systems,
such as external databases, and converting it into a savepoint.</li>
<li>State in savepoints can be queried using any of Flink’s batch APIs
(DataSet, Table, SQL), for example to analyze relevant state patterns or
check for discrepancies in state that can support application auditing or
troubleshooting.</li>
<li>The schema of state in savepoints can be migrated offline, compared to the
previous approach requiring online migration on schema access.</li>
<li>Invalid data in savepoints can be identified and corrected.</li>
</ul>
<p>The new State Processor API covers all variations of snapshots: savepoints,
full checkpoints and incremental checkpoints. More details are available in
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-43%3A&#43;State&#43;Processor&#43;API">FLIP-43</a></p>
<h3 id="stop-with-savepoint-flip-34">
Stop-with-Savepoint (FLIP-34)
<a class="anchor" href="#stop-with-savepoint-flip-34">#</a>
</h3>
<p><a href="//nightlies.apache.org/flink/flink-docs-stable/ops/state/savepoints.html#operations">Cancelling with a
savepoint</a>
is a common operation for stopping/restarting, forking or updating Flink jobs.
However, the existing implementation did not guarantee output persistence to
external storage systems for exactly-once sinks. To improve the end-to-end
semantics when stopping a job, Flink 1.9 introduces a new <code>SUSPEND</code> mode to
stop a job with a savepoint that is consistent with the emitted data.
You can suspend a job with Flink’s CLI client as follows:</p>
<pre tabindex="0"><code>bin/flink stop -p [:targetDirectory] :jobId
</code></pre><p>The final job state is set to <code>FINISHED</code> on success, allowing
users to detect failures of the requested operation.</p>
<p>More details are available in
<a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=103090212">FLIP-34</a></p>
<h3 id="flink-webui-rework">
Flink WebUI Rework
<a class="anchor" href="#flink-webui-rework">#</a>
</h3>
<p>After a
<a href="http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Change-underlying-Frontend-Architecture-for-Flink-Web-Dashboard-td24902.html">discussion</a>
about modernizing the internals of Flink’s WebUI, this component was
reconstructed using the latest stable version of Angular — basically, a bump
from Angular 1.x to 7.x. The redesigned version is the default in 1.9.0,
however there is a link to switch to the old WebUI.</p>
<div class="row"> <div class="col-sm-6"> <span><img class="thumbnail"
src="/img/blog/release-19-web1.png" /></span> </div> <div
class="col-sm-6"> <span><img class="thumbnail"
src="/img/blog/release-19-web2.png" /></span> </div>
</div>
<p><strong>Note:</strong> Moving forward, feature parity for the old version of the WebUI
will not be guaranteed.</p>
<h3 id="preview-of-the-new-blink-sql-query-processor">
Preview of the new Blink SQL Query Processor
<a class="anchor" href="#preview-of-the-new-blink-sql-query-processor">#</a>
</h3>
<p>Following the <a href="/news/2019/02/13/unified-batch-streaming-blink.html">donation of
Blink</a> to
Apache Flink, the community worked on integrating Blink’s query optimizer and
runtime for the Table API and SQL. As a first step, we refactored the
monolithic <code>flink-table</code> module into smaller modules
(<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-32%3A&#43;Restructure&#43;flink-table&#43;for&#43;future&#43;contributions">FLIP-32</a>).
This resulted in a clear separation of and well-defined interfaces between the
Java and Scala API modules and the optimizer and runtime modules.</p>
<p><span><img style="width:50%"
src="/img/blog/release-19-stack.png" /></span></p>
<p>Next, we extended Blink’s planner to implement the new optimizer interface
such that there are now two pluggable query processors to execute Table API
and SQL statements: the pre-1.9 Flink processor and the new Blink-based query
processor. The Blink-based query processor offers better SQL coverage (full TPC-H
coverage in 1.9, TPC-DS coverage is planned for the next release) and improved
performance for batch queries as the result of more extensive query
optimization (cost-based plan selection and more optimization rules), improved
code-generation, and tuned operator implementations.
The Blink-based query processor also provides a more powerful streaming runner,
with some new features (e.g. dimension table join, TopN, deduplication) and
optimizations to solve data-skew in aggregation and more useful built-in
functions.</p>
<p><strong>Note:</strong> The semantics and set of supported operations of the query
processors are mostly, but not fully aligned.</p>
<p>However, the integration of Blink’s query processor is not fully completed
yet. Therefore, the pre-1.9 Flink processor is still the default processor in
Flink 1.9 and recommended for production settings. You can enable the Blink
processor by configuring it via the <code>EnvironmentSettings</code> when creating a
<code>TableEnvironment</code>. The selected processor must be on the classpath of the
executing Java process. For cluster setups, both query processors are
automatically loaded with the default configuration. When running a query from
your IDE you need to explicitly <a href="//nightlies.apache.org/flink/flink-docs-release-1.9/dev/table/#table-program-dependencies">add a planner
dependency</a>
to your project.</p>
<h4 id="other-improvements-to-the-table-api-and-sql">
<strong>Other Improvements to the Table API and SQL</strong>
<a class="anchor" href="#other-improvements-to-the-table-api-and-sql">#</a>
</h4>
<p>Besides the exciting progress around the Blink planner, the community worked
on a whole set of other improvements to these interfaces, including:</p>
<ul>
<li>
<p><strong>Scala-free Table API and SQL for Java users
(<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-32%3A&#43;Restructure&#43;flink-table&#43;for&#43;future&#43;contributions">FLIP-32</a>)</strong></p>
<p>As part of the refactoring and splitting of the flink-table module, two
separate API modules for Java and Scala were created. For Scala users,
nothing really changes, but Java users can use the Table API and/or SQL now
without pulling in a Scala dependency.</p>
</li>
<li>
<p><strong>Rework of the Table API Type System</strong>
<strong>(<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-37%3A&#43;Rework&#43;of&#43;the&#43;Table&#43;API&#43;Type&#43;System">FLIP-37</a>)</strong></p>
<p>The community implemented a <a href="//nightlies.apache.org/flink/flink-docs-release-1.9/dev/table/types.html#data-types">new data type
system</a>
to detach the Table API from Flink’s
<a href="//nightlies.apache.org/flink/flink-docs-release-1.9/dev/types_serialization.html#flinks-typeinformation-class">TypeInformation</a>
class and improve its compliance with the SQL standard. This is still a
work in progress and expected to be completed in the next release. In
Flink 1.9, UDFs are―among other things―not ported to the new type system
yet.</p>
</li>
<li>
<p><strong>Multi-column and Multi-row Transformations for Table API</strong>
<strong>(<a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=97552739">FLIP-29</a>)</strong></p>
<p>The functionality of the Table API was extended with a set of
transformations that support multi-row and/or multi-column inputs and
outputs. These transformations significantly ease the implementation of
processing logic that would be cumbersome to implement with relational
operators.</p>
</li>
<li>
<p><strong>New, Unified Catalog APIs</strong>
<strong>(<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-30%3A&#43;Unified&#43;Catalog&#43;APIs">FLIP-30</a>)</strong></p>
<p>We reworked the catalog APIs to store metadata and unified the handling of
internal and external catalogs. This effort was mainly initiated as a
prerequisite for the Hive integration (see below), but improves the overall
convenience of managing catalog metadata in Flink. Besides improving the
catalog interfaces, we also extended their functionality. Previously table
definitions for Table API or SQL queries were volatile. With Flink 1.9, the
metadata of tables which are registered with a SQL DDL statement can be
persisted in a catalog. This means you can add a table that is backed by a
Kafka topic to a Metastore catalog and from then on query this table
whenever your catalog is connected to Metastore.</p>
</li>
<li>
<p><strong>DDL Support in the SQL API
(<a href="https://issues.apache.org/jira/browse/FLINK-10232">FLINK-10232</a>)</strong></p>
<p>Up to this point, Flink SQL only supported DML statements (e.g. <code>SELECT</code>,
<code>INSERT</code>). External tables (table sources and sinks) had to be registered
via Java/Scala code or configuration files. For 1.9, we added support for
SQL DDL statements to register and remove tables and views (<code>CREATE TABLE, DROP TABLE)</code>. However, we did not add
stream-specific syntax extensions to define timestamp extraction and
watermark generation, yet. Full support for streaming use cases is planned
for the next release.</p>
</li>
</ul>
<h3 id="preview-of-full-hive-integration-flink-10556">
Preview of Full Hive Integration (FLINK-10556)
<a class="anchor" href="#preview-of-full-hive-integration-flink-10556">#</a>
</h3>
<p>Apache Hive is widely used in Hadoop’s ecosystem to store and query large
amounts of structured data. Besides being a query processor, Hive features a
catalog called Metastore to manage and organize large datasets. A common
integration point for query processors is to integrate with Hive’s Metastore
in order to be able to tap into the data managed by Hive.</p>
<p>Recently, the community started implementing an external catalog for Flink’s
Table API and SQL that connects to Hive’s Metastore. In Flink 1.9, users will
be able to query and process all data that is stored in Hive. As described
earlier, you will also be able to persist metadata of Flink tables in Metastore.
Moreover, the Hive integration includes support to use Hive’s UDFs in Flink
Table API or SQL queries. More details are available in
<a href="https://issues.apache.org/jira/browse/FLINK-10556">FLINK-10556</a>.</p>
<p>While, previously, table definitions for Table API or SQL queries were always
volatile, the new catalog connector additionally allows persisting a table in
Metastore that is created with a SQL DDL statement (see above). This means
that you connect to Metastore and register a table that is, for example,
backed by a Kafka topic. From now on, you can query that table whenever your
catalog is connected to Metastore.</p>
<p>Please note that the Hive support in Flink 1.9 is experimental. We are
planning to stabilize these features for the next release and are looking
forward to your feedback.</p>
<h3 id="preview-of-the-new-python-table-api-flip-38">
Preview of the new Python Table API (FLIP-38)
<a class="anchor" href="#preview-of-the-new-python-table-api-flip-38">#</a>
</h3>
<p>This release also introduces a first version of a Python Table API
(<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-38%3A&#43;Python&#43;Table&#43;API">FLIP-38</a>).
This marks the start towards our goal of bringing
full-fledged Python support to Flink. The feature was designed as a slim
Python API wrapper around the Table API, basically translating Python Table
API method calls into Java Table API calls. In the initial version that ships
with Flink 1.9, the Python Table API does not support UDFs yet, but just
standard relational operations. Support for UDFs implemented in Python is on
the roadmap for future releases.</p>
<p>If you’d like to try the new Python API, you have to manually <a href="//nightlies.apache.org/flink/flink-docs-release-1.9/flinkDev/building.html#build-pyflink">install
PyFlink</a>.
From there, you can have a look at <a href="//nightlies.apache.org/flink/flink-docs-release-1.9/tutorials/python_table_api.html">this
walkthrough</a>
or explore it on your own. The <a href="http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/VOTE-Publish-the-PyFlink-into-PyPI-td31201.html">community is currently
working</a>
on preparing a <code>pyflink</code> Python package that will be made available for
installation via <code>pip</code>.</p>
<h2 id="important-changes">
Important Changes
<a class="anchor" href="#important-changes">#</a>
</h2>
<ul>
<li>The Table API and SQL are now part of the default configuration of the
Flink distribution. Before, the Table API and SQL had to be enabled by
moving the corresponding JAR file from ./opt to ./lib.</li>
<li>The machine learning library (flink-ml) has been removed in preparation for
<a href="https://docs.google.com/document/d/1StObo1DLp8iiy0rbukx8kwAJb0BwDZrQrMWub3DzsEo/edit">FLIP-39</a>.</li>
<li>The old DataSet and DataStream Python APIs have been removed in favor of
<a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-38%3A&#43;Python&#43;Table&#43;API">FLIP-38</a>.</li>
<li>Flink can be compiled and run on Java 9. Note that certain components
interacting with external systems (connectors, filesystems, reporters) may
not work since the respective projects may have skipped Java 9 support.</li>
</ul>
<h2 id="release-notes">
Release Notes
<a class="anchor" href="#release-notes">#</a>
</h2>
<p>Please review the <a href="//nightlies.apache.org/flink/flink-docs-release-1.9/release-notes/flink-1.9.html">release
notes</a>
for a more detailed list of changes and new features if you plan to upgrade
your Flink setup to Flink 1.9.0.</p>
<h2 id="list-of-contributors">
List of Contributors
<a class="anchor" href="#list-of-contributors">#</a>
</h2>
<p>We would like to thank all contributors who have made this release possible:</p>
<p>Abdul Qadeer (abqadeer), Aitozi, Alberto Romero, Aleksey Pak, Alexander
Fedulov, Alice Yan, Aljoscha Krettek, Aloys, Andrew Duffy, Andrey Zagrebin,
Ankur, Artsem Semianenka, Benchao Li, Biao Liu, Bo WANG, Bowen L, Chesnay
Schepler, Clark Yang, Congxian Qiu, Cristian, Danny Chan, David Moravek, Dawid
Wysakowicz, Dian Fu, EronWright, Fabian Hueske, Fabio Lombardelli, Fokko
Driesprong, Gao Yun, Gary Yao, Gen Luo, Gyula Fora, Hequn Cheng,
Hongtao Zhang, Huang Xingbo, HuangXingBo, Hugo Da Cruz Louro, Humberto
Rodríguez A, Hwanju Kim, Igal Shilman, Jamie Grier, Jark Wu, Jason, Jasper
Yue, Jeff Zhang, Jiangjie (Becket) Qin, Jiezhi.G, Jincheng Sun, Jing Zhang,
Jingsong Lee, Juan Gentile, Jungtaek Lim, Kailash Dayanand, Kevin
Bohinski, Konstantin Knauf, Konstantinos Papadopoulos, Kostas Kloudas, Kurt
Young, Lakshmi, Lakshmi Gururaja Rao, Leeviiii, LouisXu, Maximilian Michels,
Nico Kruber, Niels Basjes, Paul Lam, PengFei Li, Peter Huang, Pierre Zemb,
Piotr Nowojski, Piyush Narang, Richard Deurwaarder, Robert Metzger, Robert
Stoll, Romano Vacca, Rong Rong, Rui Li, Ryantaocer, Scott Mitchell, Seth
Wiesman, Shannon Carey, Shimin Yang, Stefan Richter, Stephan Ewen, Stephen
Connolly, Steven Wu, SuXingLee, TANG Wen-hui, Thomas Weise, Till Rohrmann,
Timo Walther, Tom Goong, TsReaper, Tzu-Li (Gordon) Tai, Ufuk Celebi,
Victor Wong, WangHengwei, Wei Zhong, WeiZhong94, Xintong Song, Xpray,
XuQianJin-Stars, Xuefu Zhang, Xupingyong, Yangze Guo, Yu Li, Yun Gao, Yun
Tang, Zhanchun Zhang, Zhenghua Gao, Zhijiang, Zhu Zhu, Zili
Chen, aloys, arganzheng, azagrebin, bd2019us, beyond1920, biao.liub,
blueszheng, boshu Zheng, chenqi, chummyhe89, chunpinghe, dcadmin,
dianfu, godfrey he, guanghui01.rong, hehuiyuan, hello, hequn8128,
jackyyin, joongkeun.yang, klion26, lamber-ken, leesf, liguowei,
lincoln-lil, liyafan82, luoqi, mans2singh, maqingxiang, maxin, mjl, okidogi,
ozan, potseluev, qiangsi.lq, qiaoran, robbinli, shaoxuan-wang, shengqian.zhou,
shenlang.sl, shuai-xu, sunhaibotb, tianchen, tianchen92,
tison, tom_gong, vinoyang, vthinkxie, wanggeng3, wenhuitang, winifredtamg,
xl38154, xuyang1706, yangfei5, yanghua, yuzhao.cyz,
zhangxin516, zhangxinxing, zhaofaxian, zhijiang, zjuwangg, 林小铂,
黄培松, 时无两丶.</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/2019-08-22-release-1.9.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>
<ul>
<li><a href="#new-features-and-improvements">New Features and Improvements</a>
<ul>
<li><a href="#fine-grained-batch-recovery-flip-1">Fine-grained Batch Recovery (FLIP-1)</a></li>
<li><a href="#state-processor-api-flip-43">State Processor API (FLIP-43)</a></li>
<li><a href="#stop-with-savepoint-flip-34">Stop-with-Savepoint (FLIP-34)</a></li>
<li><a href="#flink-webui-rework">Flink WebUI Rework</a></li>
<li><a href="#preview-of-the-new-blink-sql-query-processor">Preview of the new Blink SQL Query Processor</a></li>
<li><a href="#preview-of-full-hive-integration-flink-10556">Preview of Full Hive Integration (FLINK-10556)</a></li>
<li><a href="#preview-of-the-new-python-table-api-flip-38">Preview of the new Python Table API (FLIP-38)</a></li>
</ul>
</li>
<li><a href="#important-changes">Important Changes</a></li>
<li><a href="#release-notes">Release Notes</a></li>
<li><a href="#list-of-contributors">List of Contributors</a></li>
</ul>
</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>