| <!DOCTYPE html> |
| <html lang="en"> |
| <head> |
| <meta charset="utf-8"> |
| <meta http-equiv="X-UA-Compatible" content="IE=edge"> |
| <meta name="viewport" content="width=device-width, initial-scale=1"> |
| <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags --> |
| <title>Apache Flink: Apache Flink 1.4.0 Release Announcement</title> |
| <link rel="shortcut icon" href="/favicon.ico" type="image/x-icon"> |
| <link rel="icon" href="/favicon.ico" type="image/x-icon"> |
| |
| <!-- Bootstrap --> |
| <link rel="stylesheet" href="https://maxcdn.bootstrapcdn.com/bootstrap/3.4.1/css/bootstrap.min.css"> |
| <link rel="stylesheet" href="/css/flink.css"> |
| <link rel="stylesheet" href="/css/syntax.css"> |
| |
| <!-- Blog RSS feed --> |
| <link href="/blog/feed.xml" rel="alternate" type="application/rss+xml" title="Apache Flink Blog: RSS feed" /> |
| |
| <!-- jQuery (necessary for Bootstrap's JavaScript plugins) --> |
| <!-- We need to load Jquery in the header for custom google analytics event tracking--> |
| <script src="/js/jquery.min.js"></script> |
| |
| <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries --> |
| <!-- WARNING: Respond.js doesn't work if you view the page via file:// --> |
| <!--[if lt IE 9]> |
| <script src="https://oss.maxcdn.com/html5shiv/3.7.2/html5shiv.min.js"></script> |
| <script src="https://oss.maxcdn.com/respond/1.4.2/respond.min.js"></script> |
| <![endif]--> |
| </head> |
| <body> |
| |
| |
| <!-- Main content. --> |
| <div class="container"> |
| <div class="row"> |
| |
| |
| <div id="sidebar" class="col-sm-3"> |
| |
| |
| <!-- Top navbar. --> |
| <nav class="navbar navbar-default"> |
| <!-- The logo. --> |
| <div class="navbar-header"> |
| <button type="button" class="navbar-toggle collapsed" data-toggle="collapse" data-target="#bs-example-navbar-collapse-1"> |
| <span class="icon-bar"></span> |
| <span class="icon-bar"></span> |
| <span class="icon-bar"></span> |
| </button> |
| <div class="navbar-logo"> |
| <a href="/"> |
| <img alt="Apache Flink" src="/img/flink-header-logo.svg" width="147px" height="73px"> |
| </a> |
| </div> |
| </div><!-- /.navbar-header --> |
| |
| <!-- The navigation links. --> |
| <div class="collapse navbar-collapse" id="bs-example-navbar-collapse-1"> |
| <ul class="nav navbar-nav navbar-main"> |
| |
| <!-- First menu section explains visitors what Flink is --> |
| |
| <!-- What is Stream Processing? --> |
| <!-- |
| <li><a href="/streamprocessing1.html">What is Stream Processing?</a></li> |
| --> |
| |
| <!-- What is Flink? --> |
| <li><a href="/flink-architecture.html">What is Apache Flink?</a></li> |
| |
| |
| |
| <!-- What is Stateful Functions? --> |
| |
| <li><a href="/stateful-functions.html">What is Stateful Functions?</a></li> |
| |
| <!-- Use cases --> |
| <li><a href="/usecases.html">Use Cases</a></li> |
| |
| <!-- Powered by --> |
| <li><a href="/poweredby.html">Powered By</a></li> |
| |
| |
| |
| <!-- Second menu section aims to support Flink users --> |
| |
| <!-- Downloads --> |
| <li><a href="/downloads.html">Downloads</a></li> |
| |
| <!-- Getting Started --> |
| <li class="dropdown"> |
| <a class="dropdown-toggle" data-toggle="dropdown" href="#">Getting Started<span class="caret"></span></a> |
| <ul class="dropdown-menu"> |
| <li><a href="https://ci.apache.org/projects/flink/flink-docs-release-1.11/getting-started/index.html" target="_blank">With Flink <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| <li><a href="https://ci.apache.org/projects/flink/flink-statefun-docs-release-2.1/getting-started/project-setup.html" target="_blank">With Flink Stateful Functions <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| <li><a href="/training.html">Training Course</a></li> |
| </ul> |
| </li> |
| |
| <!-- Documentation --> |
| <li class="dropdown"> |
| <a class="dropdown-toggle" data-toggle="dropdown" href="#">Documentation<span class="caret"></span></a> |
| <ul class="dropdown-menu"> |
| <li><a href="https://ci.apache.org/projects/flink/flink-docs-release-1.11" target="_blank">Flink 1.11 (Latest stable release) <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| <li><a href="https://ci.apache.org/projects/flink/flink-docs-master" target="_blank">Flink Master (Latest Snapshot) <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| <li><a href="https://ci.apache.org/projects/flink/flink-statefun-docs-release-2.1" target="_blank">Flink Stateful Functions 2.1 (Latest stable release) <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| <li><a href="https://ci.apache.org/projects/flink/flink-statefun-docs-master" target="_blank">Flink Stateful Functions Master (Latest Snapshot) <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| </ul> |
| </li> |
| |
| <!-- getting help --> |
| <li><a href="/gettinghelp.html">Getting Help</a></li> |
| |
| <!-- Blog --> |
| <li class="active"><a href="/blog/"><b>Flink Blog</b></a></li> |
| |
| |
| <!-- Flink-packages --> |
| <li> |
| <a href="https://flink-packages.org" target="_blank">flink-packages.org <small><span class="glyphicon glyphicon-new-window"></span></small></a> |
| </li> |
| |
| |
| <!-- Third menu section aim to support community and contributors --> |
| |
| <!-- Community --> |
| <li><a href="/community.html">Community & Project Info</a></li> |
| |
| <!-- Roadmap --> |
| <li><a href="/roadmap.html">Roadmap</a></li> |
| |
| <!-- Contribute --> |
| <li><a href="/contributing/how-to-contribute.html">How to Contribute</a></li> |
| |
| |
| <!-- GitHub --> |
| <li> |
| <a href="https://github.com/apache/flink" target="_blank">Flink on GitHub <small><span class="glyphicon glyphicon-new-window"></span></small></a> |
| </li> |
| |
| |
| |
| <!-- Language Switcher --> |
| <li> |
| |
| |
| <!-- link to the Chinese home page when current is blog page --> |
| <a href="/zh">中文版</a> |
| |
| |
| </li> |
| |
| </ul> |
| |
| <ul class="nav navbar-nav navbar-bottom"> |
| <hr /> |
| |
| <!-- Twitter --> |
| <li><a href="https://twitter.com/apacheflink" target="_blank">@ApacheFlink <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| |
| <!-- Visualizer --> |
| <li class=" hidden-md hidden-sm"><a href="/visualizer/" target="_blank">Plan Visualizer <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| |
| <hr /> |
| |
| <li><a href="https://apache.org" target="_blank">Apache Software Foundation <small><span class="glyphicon glyphicon-new-window"></span></small></a></li> |
| |
| <li> |
| <style> |
| .smalllinks:link { |
| display: inline-block !important; background: none; padding-top: 0px; padding-bottom: 0px; padding-right: 0px; min-width: 75px; |
| } |
| </style> |
| |
| <a class="smalllinks" href="https://www.apache.org/licenses/" target="_blank">License</a> <small><span class="glyphicon glyphicon-new-window"></span></small> |
| |
| <a class="smalllinks" href="https://www.apache.org/security/" target="_blank">Security</a> <small><span class="glyphicon glyphicon-new-window"></span></small> |
| |
| <a class="smalllinks" href="https://www.apache.org/foundation/sponsorship.html" target="_blank">Donate</a> <small><span class="glyphicon glyphicon-new-window"></span></small> |
| |
| <a class="smalllinks" href="https://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a> <small><span class="glyphicon glyphicon-new-window"></span></small> |
| </li> |
| |
| </ul> |
| </div><!-- /.navbar-collapse --> |
| </nav> |
| |
| </div> |
| <div class="col-sm-9"> |
| <div class="row-fluid"> |
| <div class="col-sm-12"> |
| <div class="row"> |
| <h1>Apache Flink 1.4.0 Release Announcement</h1> |
| <p><i></i></p> |
| |
| <article> |
| <p>12 Dec 2017 Aljoscha Krettek (<a href="https://twitter.com/aljoscha">@aljoscha</a>) & Mike Winters (<a href="https://twitter.com/wints">@wints</a>)</p> |
| |
| <p>The Apache Flink community is pleased to announce the 1.4.0 release. Over the past 5 months, the |
| Flink community has been working hard to resolve more than 900 issues. See the <a href="https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522&version=12340533">complete changelog</a> |
| for more detail.</p> |
| |
| <p>This is the fifth major release in the 1.x.y series. It is API-compatible with the other 1.x.y |
| releases for APIs annotated with the @Public annotation.</p> |
| |
| <p>We encourage everyone to download the release and check out the <a href="https://ci.apache.org/projects/flink/flink-docs-release-1.4/">documentation</a>.</p> |
| |
| <p>Feedback through the <a href="http://flink.apache.org/community.html#mailing-lists">Flink mailing lists</a> is, as always, gladly encouraged!</p> |
| |
| <p>You can find the binaries on the updated <a href="http://flink.apache.org/downloads.html">Downloads</a> page on the Flink project site.</p> |
| |
| <p>The release includes improvements to many different aspects of Flink, including:</p> |
| |
| <ul> |
| <li>The ability to build end-to-end exactly-once applications with Flink and popular data sources and sinks such as Apache Kafka.</li> |
| <li>A more developer-friendly dependency structure as well as Hadoop-free Flink for Flink users who do not have Hadoop dependencies.</li> |
| <li>Support for JOIN and for new sources and sinks in table API and SQL, expanding the range of logic that can be expressed with these APIs.</li> |
| </ul> |
| |
| <p>A summary of some of the features in the release is available below.</p> |
| |
| <p>For more background on the Flink 1.4.0 release and the work planned for the Flink 1.5.0 release, please refer to <a href="http://flink.apache.org/news/2017/11/22/release-1.4-and-1.5-timeline.html">this blog post</a> on the Apache Flink blog.</p> |
| |
| <div class="page-toc"> |
| <ul id="markdown-toc"> |
| <li><a href="#new-features-and-improvements" id="markdown-toc-new-features-and-improvements">New Features and Improvements</a> <ul> |
| <li><a href="#end-to-end-exactly-once-applications-with-apache-flink-and-apache-kafka-and-twophasecommitsinkfunction" id="markdown-toc-end-to-end-exactly-once-applications-with-apache-flink-and-apache-kafka-and-twophasecommitsinkfunction">End-to-end Exactly Once Applications with Apache Flink and Apache Kafka and TwoPhaseCommitSinkFunction</a></li> |
| <li><a href="#table-api-and-streaming-sql-enhancements" id="markdown-toc-table-api-and-streaming-sql-enhancements">Table API and Streaming SQL Enhancements</a></li> |
| <li><a href="#a-significantly-improved-dependency-structure-and-reversed-class-loading" id="markdown-toc-a-significantly-improved-dependency-structure-and-reversed-class-loading">A Significantly-Improved Dependency Structure and Reversed Class Loading</a></li> |
| <li><a href="#hadoop-free-flink" id="markdown-toc-hadoop-free-flink">Hadoop-free Flink</a></li> |
| <li><a href="#improvements-to-flink-internals" id="markdown-toc-improvements-to-flink-internals">Improvements to Flink Internals</a></li> |
| <li><a href="#improvements-to-the-queryable-state-client" id="markdown-toc-improvements-to-the-queryable-state-client">Improvements to the Queryable State Client</a></li> |
| <li><a href="#metrics-and-monitoring" id="markdown-toc-metrics-and-monitoring">Metrics and Monitoring</a></li> |
| <li><a href="#connector-improvements-and-fixes" id="markdown-toc-connector-improvements-and-fixes">Connector improvements and fixes</a></li> |
| </ul> |
| </li> |
| <li><a href="#release-notes---please-read" id="markdown-toc-release-notes---please-read">Release Notes - Please Read</a> <ul> |
| <li><a href="#changes-to-dynamic-class-loading-of-user-code" id="markdown-toc-changes-to-dynamic-class-loading-of-user-code">Changes to dynamic class loading of user code</a></li> |
| <li><a href="#no-more-avro-dependency-included-by-default" id="markdown-toc-no-more-avro-dependency-included-by-default">No more Avro dependency included by default</a></li> |
| <li><a href="#hadoop-free-flink-1" id="markdown-toc-hadoop-free-flink-1">Hadoop-free Flink</a></li> |
| <li><a href="#bundled-s3-filesystems" id="markdown-toc-bundled-s3-filesystems">Bundled S3 FileSystems</a></li> |
| </ul> |
| </li> |
| <li><a href="#list-of-contributors" id="markdown-toc-list-of-contributors">List of Contributors</a></li> |
| </ul> |
| |
| </div> |
| |
| <h2 id="new-features-and-improvements">New Features and Improvements</h2> |
| |
| <h3 id="end-to-end-exactly-once-applications-with-apache-flink-and-apache-kafka-and-twophasecommitsinkfunction">End-to-end Exactly Once Applications with Apache Flink and Apache Kafka and TwoPhaseCommitSinkFunction</h3> |
| |
| <p>Flink 1.4 includes a first version of an exactly-once producer for Apache Kafka 0.11. This producer |
| enables developers who build Flink applications with Kafka as a data source and sink to compute |
| exactly-once results not just within the Flink program, but truly “end-to-end” in the application.</p> |
| |
| <p>The common pattern used for exactly-once applications in Kafka and in other sinks–the two-phase |
| commit algorithm–has been extracted in Flink 1.4.0 into a common class, the |
| TwoPhaseCommitSinkFunction (<a href="https://issues.apache.org/jira/browse/FLINK-7210">FLINK-7210</a>). This |
| will make it easier for users to create their own exactly-once data sinks in the future.</p> |
| |
| <h3 id="table-api-and-streaming-sql-enhancements">Table API and Streaming SQL Enhancements</h3> |
| |
| <p>Flink SQL now supports windowed joins based on processing time and event time |
| (<a href="https://issues.apache.org/jira/browse/FLINK-5725">FLINK-5725</a>). Users will be able to execute a |
| join between 2 streaming tables and compute windowed results according to these 2 different concepts |
| of time. The syntax and semantics in Flink are the same as standard SQL with JOIN and with Flink’s |
| streaming SQL more broadly.</p> |
| |
| <p>Flink SQL also now supports “INSERT INTO SELECT” queries, which makes it possible to write results |
| from SQL directly into a data sink (an external system that receives data from a Flink application). |
| This improves operability and ease-of-use of Flink SQL.</p> |
| |
| <p>The Table API now supports aggregations on streaming tables; previously, the only supported |
| operations on streaming tables were projection, selection, and union |
| (<a href="https://issues.apache.org/jira/browse/FLINK-4557">FLINK-4557</a>). This feature was initially discussed in Flink |
| Improvement Proposal 11: <a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-11%3A+Table+API+Stream+Aggregations">FLIP-11</a>.</p> |
| |
| <p>The release also adds support for new table API and SQL sources and sinks, including a Kafka 0.11 |
| source and JDBC sink.</p> |
| |
| <p>Lastly, Flink SQL now uses Apache Calcite 1.14, which was just released in October 2017 |
| (<a href="https://issues.apache.org/jira/browse/FLINK-7051">FLINK-7051</a>).</p> |
| |
| <h3 id="a-significantly-improved-dependency-structure-and-reversed-class-loading">A Significantly-Improved Dependency Structure and Reversed Class Loading</h3> |
| |
| <p>Flink 1.4.0 shades a number of dependences and subtle runtime conflicts, including:</p> |
| |
| <ul> |
| <li>ASM</li> |
| <li>Guava</li> |
| <li>Jackson</li> |
| <li>Netty</li> |
| <li>Apache Zookeeper</li> |
| </ul> |
| |
| <p>These changes improve Flink’s overall stability and removes friction when embedding Flink or calling |
| Flink “library style”.</p> |
| |
| <p>The release also introduces default reversed (child-first) class loading for dynamically-loaded user |
| code, allowing for different dependencies than those included in the core framework.</p> |
| |
| <p>For details on those changes please check out the relevant Jira issues:</p> |
| |
| <ul> |
| <li><a href="https://issues.apache.org/jira/browse/FLINK-7442">FLINK-7442</a></li> |
| <li><a href="https://issues.apache.org/jira/browse/FLINK-6529">FLINK-6529</a></li> |
| </ul> |
| |
| <h3 id="hadoop-free-flink">Hadoop-free Flink</h3> |
| |
| <p>Apache Flink users without any Apache Hadoop dependencies can now run Flink without Hadoop. Flink |
| programs that do not rely on Hadoop components can now be much smaller, a benefit particularly in a |
| container-based setup resulting in less network traffic and better performance.</p> |
| |
| <p>This includes the addition of Flink’s own Amazon S3 filesystem implementations based on Hadoop’s S3a |
| and Presto’s S3 file system with properly shaded dependencies (<a href="https://issues.apache.org/jira/browse/FLINK-5706">FLINK-5706</a>).</p> |
| |
| <p>The details of these changes regarding Hadoop-free Flink are available in the Jira issue: |
| <a href="https://issues.apache.org/jira/browse/FLINK-2268">FLINK-2268</a>.</p> |
| |
| <h3 id="improvements-to-flink-internals">Improvements to Flink Internals</h3> |
| |
| <p>Flink 1.4.0 introduces a new blob storage architecture that was first discussed in |
| <a href="https://cwiki.apache.org/confluence/display/FLINK/FLIP-19%3A+Improved+BLOB+storage+architecture">Flink Improvement Proposal 19</a> (<a href="https://issues.apache.org/jira/browse/FLINK-6916">FLINK-6916</a>).</p> |
| |
| <p>This will enable easier integration with both the work being done in <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=65147077">Flink Improvement Proposal 6</a> in |
| the future and with other improvements in the 1.4.0 release, such as support for messages larger |
| than the maximum Akka Framesize (<a href="https://issues.apache.org/jira/browse/FLINK-6046">FLINK-6046</a>).</p> |
| |
| <p>The improvement also enables Flink to leverage distributed file systems in high availability |
| settings for optimized distribution of deployment data to TaskManagers.</p> |
| |
| <h3 id="improvements-to-the-queryable-state-client">Improvements to the Queryable State Client</h3> |
| |
| <p>Flink’s <a href="https://ci.apache.org/projects/flink/flink-docs-release-1.4/dev/stream/state/queryable_state.html">queryable state</a> makes it possible for users to access application state directly in Flink |
| before the state has been sent to an external database or key-value store.</p> |
| |
| <p>Flink 1.4.0 introduces a range of improvements to the queryable state client, including a more |
| container-friendly architecture, a more user-friendly API that hides configuration parameters, and |
| the groundwork to be able to expose window state (the state of an in-flight window) in the future.</p> |
| |
| <p>For details about the changes to queryable state please refer to the umbrella Jira issue: |
| <a href="https://issues.apache.org/jira/browse/FLINK-5675">FLINK-5675</a>.</p> |
| |
| <h3 id="metrics-and-monitoring">Metrics and Monitoring</h3> |
| |
| <p>Flink’s metrics system now also includes support for Prometheus, an increasingly-popular metrics and |
| reporting system within the Flink community (<a href="https://issues.apache.org/jira/browse/FLINK-6221">FLINK-6221</a>).</p> |
| |
| <p>And the Apache Kafka connector in Flink now exposes metrics for failed and successful offset commits |
| in the Kafka consumer callback (<a href="https://issues.apache.org/jira/browse/FLINK-6998">FLINK-6998</a>).</p> |
| |
| <h3 id="connector-improvements-and-fixes">Connector improvements and fixes</h3> |
| |
| <p>Flink 1.4.0 introduces an Apache Kafka 0.11 connector and, as described above, support for an |
| exactly-once producer for Kafka 0.11 (<a href="https://issues.apache.org/jira/browse/FLINK-6988">FLINK-6988</a>).</p> |
| |
| <p>Additionally, the Flink-Kafka consumer now supports dynamic partition discovery & topic discovery |
| based on regex. This means that the Flink-Kafka consumer can pick up new Kafka partitions without |
| needing to restart the job and while maintaining exactly-once guarantees |
| (<a href="https://issues.apache.org/jira/browse/FLINK-4022">FLINK-4022</a>).</p> |
| |
| <p>Flink’s Apache Kinesis connector now uses an updated version of the Kinesis Consumer Library and |
| Kinesis Consumer Library. This introduces improved retry logic to the connector and should |
| significantly reduce the number of failures caused by Flink writing too quickly to Kinesis |
| (<a href="https://issues.apache.org/jira/browse/FLINK-7366">FLINK-7366</a>).</p> |
| |
| <p>Flink’s Apache Cassandra connector now supports Scala tuples–previously, only streams of Java |
| tuples were supported (<a href="https://issues.apache.org/jira/browse/FLINK-4497">FLINK-4497</a>). Also, a bug was fixed in |
| the Cassandra connector that caused messages to be lost in certain instances |
| (<a href="https://issues.apache.org/jira/browse/FLINK-4500">FLINK-4500</a>).</p> |
| |
| <h2 id="release-notes---please-read">Release Notes - Please Read</h2> |
| |
| <p>Some of these changes will require updating the configuration or Maven dependencies for existing |
| programs. Please read below to see if you might be affected.</p> |
| |
| <h3 id="changes-to-dynamic-class-loading-of-user-code">Changes to dynamic class loading of user code</h3> |
| |
| <p>As mentioned above, we changed the way Flink loads user code from the previous default of |
| <em>parent-first class loading</em> (the default for Java) to <em>child-first classloading</em>, which is a common |
| practice in Java Application Servers, where this is also referred to as inverted or reversed class |
| loading.</p> |
| |
| <p>This should not affect regular user code but will enable programs to use a different version of |
| dependencies that come with Flink – for example Akka, netty, or Jackson. If you want to change back |
| to the previous default, you can use the configuration setting <code>classloader.resolve-order: parent-first</code>, |
| the new default being <code>child-first</code>.</p> |
| |
| <h3 id="no-more-avro-dependency-included-by-default">No more Avro dependency included by default</h3> |
| |
| <p>Flink previously included Avro by default so user programs could simply use Avro and not worry about |
| adding any dependencies. This behavior was changed in Flink 1.4 because it can lead to dependency |
| clashes.</p> |
| |
| <p>You now must manually include the Avro dependency (<code>flink-avro</code>) with your program jar (or add it to |
| the Flink lib folder) if you want to use Avro.</p> |
| |
| <h3 id="hadoop-free-flink-1">Hadoop-free Flink</h3> |
| |
| <p>Starting with version 1.4, Flink can run without any Hadoop dependencies present in the Classpath. |
| Along with simply running without Hadoop, this enables Flink to dynamically use whatever Hadoop |
| version is available in the classpath.</p> |
| |
| <p>You could, for example, download the Hadoop-free release of Flink but use that to run on any |
| supported version of YARN, and Flink would dynamically use the Hadoop dependencies from YARN.</p> |
| |
| <p>This also means that in cases where you used connectors to HDFS, such as the <code>BucketingSink</code> or |
| <code>RollingSink</code>, you now have to ensure that you either use a Flink distribution with bundled Hadoop |
| dependencies or make sure to include Hadoop dependencies when building a jar file for your |
| application.</p> |
| |
| <h3 id="bundled-s3-filesystems">Bundled S3 FileSystems</h3> |
| |
| <p>Flink 1.4 comes bundled with two different S3 FileSystems based on the Presto S3 FileSystem and |
| the Hadoop S3A FileSystem. They don’t have dependencies (because all dependencies are |
| shaded/relocated) and you can use them by dropping the respective file from the <code>opt</code> directory |
| into the <code>lib</code> directory of your Flink installation. For more information about this, please refer |
| to the <a href="https://ci.apache.org/projects/flink/flink-docs-release-1.4/ops/filesystems.html#built-in-file-systems">documentation</a>.</p> |
| |
| <h2 id="list-of-contributors">List of Contributors</h2> |
| |
| <p>According to git shortlog, the following 106 people contributed to the 1.4.0 release. Thank you to |
| all contributors!</p> |
| |
| <p>Ajay Tripathy, Alejandro Alcalde, Aljoscha Krettek, Bang, Phiradet, Bowen Li, Chris Ward, Cristian, |
| Dan Kelley, David Anderson, Dawid Wysakowicz, Dian Fu, Dmitrii Kniazev, DmytroShkvyra, Fabian |
| Hueske, FlorianFan, Fokko Driesprong, Gabor Gevay, Gary Yao, Greg Hogan, Haohui Mai, Hequn Cheng, |
| James Lafa, Jark Wu, Jie Shen, Jing Fan, JingsongLi, Joerg Schad, Juan Paulo Gutierrez, Ken Geis, |
| Kent Murra, Kurt Young, Lim Chee Hau, Maximilian Bode, Michael Fong, Mike Kobit, Mikhail Lipkovich, |
| Nico Kruber, Novotnik, Petr, Nycholas de Oliveira e Oliveira, Patrick Lucas, Piotr Nowojski, Robert |
| Metzger, Rodrigo Bonifacio, Rong Rong, Scott Kidder, Sebastian Klemke, Shuyi Chen, Stefan Richter, |
| Stephan Ewen, Svend Vanderveken, Till Rohrmann, Tony Wei, Tzu-Li (Gordon) Tai, Ufuk Celebi, Usman |
| Younas, Vetriselvan1187, Vishnu Viswanath, Wright, Eron, Xingcan Cui, Xpray, Yestin, Yonatan Most, |
| Zhenzhong Xu, Zhijiang, adebski, asdf2014, bbayani, biao.liub, cactuslrd.lird, dawidwys, desktop, |
| fengyelei, godfreyhe, gosubpl, gyao, hongyuhong, huafengw, kkloudas, kl0u, lincoln-lil, |
| lingjinjiang, mengji.fy, minwenjun, mtunique, p1tz, paul, rtudoran, shaoxuan-wang, sirko |
| bretschneider, sunjincheng121, tedyu, twalthr, uybhatti, wangmiao1981, yew1eb, z00376786, zentol, |
| zhangminglei, zhe li, zhouhai02, zjureel, 付典, 军长, 宝牛, 淘江, 金竹</p> |
| |
| </article> |
| </div> |
| |
| <div class="row"> |
| <div id="disqus_thread"></div> |
| <script type="text/javascript"> |
| /* * * CONFIGURATION VARIABLES: EDIT BEFORE PASTING INTO YOUR WEBPAGE * * */ |
| var disqus_shortname = 'stratosphere-eu'; // required: replace example with your forum shortname |
| |
| /* * * DON'T EDIT BELOW THIS LINE * * */ |
| (function() { |
| var dsq = document.createElement('script'); dsq.type = 'text/javascript'; dsq.async = true; |
| dsq.src = '//' + disqus_shortname + '.disqus.com/embed.js'; |
| (document.getElementsByTagName('head')[0] || document.getElementsByTagName('body')[0]).appendChild(dsq); |
| })(); |
| </script> |
| </div> |
| </div> |
| </div> |
| </div> |
| </div> |
| |
| <hr /> |
| |
| <div class="row"> |
| <div class="footer text-center col-sm-12"> |
| <p>Copyright © 2014-2019 <a href="http://apache.org">The Apache Software Foundation</a>. All Rights Reserved.</p> |
| <p>Apache Flink, Flink®, Apache®, the squirrel logo, and the Apache feather logo are either registered trademarks or trademarks of The Apache Software Foundation.</p> |
| <p><a href="/privacy-policy.html">Privacy Policy</a> · <a href="/blog/feed.xml">RSS feed</a></p> |
| </div> |
| </div> |
| </div><!-- /.container --> |
| |
| <!-- Include all compiled plugins (below), or include individual files as needed --> |
| <script src="https://maxcdn.bootstrapcdn.com/bootstrap/3.3.4/js/bootstrap.min.js"></script> |
| <script src="https://cdnjs.cloudflare.com/ajax/libs/jquery.matchHeight/0.7.0/jquery.matchHeight-min.js"></script> |
| <script src="/js/codetabs.js"></script> |
| <script src="/js/stickysidebar.js"></script> |
| |
| <!-- Google Analytics --> |
| <script> |
| (function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){ |
| (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new Date();a=s.createElement(o), |
| m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m) |
| })(window,document,'script','//www.google-analytics.com/analytics.js','ga'); |
| |
| ga('create', 'UA-52545728-1', 'auto'); |
| ga('send', 'pageview'); |
| </script> |
| </body> |
| </html> |