blob: ded6b53ee817f05d2169e3b943f106d39e4a719e [file] [log] [blame]
<!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 Code Style and Quality Guide — Java</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>
&nbsp;
<!-- 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><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>
&nbsp;
<!-- Third menu section aim to support community and contributors -->
<!-- Community -->
<li><a href="/community.html">Community &amp; 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>
<ul class="nav navbar-nav navbar-subnav">
<li >
<a href="/contributing/contribute-code.html">Contribute Code</a>
</li>
<li >
<a href="/contributing/reviewing-prs.html">Review Pull Requests</a>
</li>
<li >
<a href="/contributing/code-style-and-quality-preamble.html">Code Style and Quality Guide</a>
</li>
<li >
<a href="/contributing/contribute-documentation.html">Contribute Documentation</a>
</li>
<li >
<a href="/contributing/docs-style.html">Documentation Style Guide</a>
</li>
<li >
<a href="/contributing/improve-website.html">Contribute to the Website</a>
</li>
</ul>
<!-- 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>
&nbsp;
<!-- Language Switcher -->
<li>
<a href="/zh/contributing/code-style-and-quality-java.html">中文版</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">
<h1>Apache Flink Code Style and Quality Guide — Java</h1>
<ul class="list-group" style="padding-top: 30px; font-weight: bold;">
<li class="list-group-item">
<a href="/contributing/code-style-and-quality-preamble.html">
Preamble
</a>
</li>
<li class="list-group-item">
<a href="/contributing/code-style-and-quality-pull-requests.html">
Pull Requests &amp; Changes
</a>
</li>
<li class="list-group-item">
<a href="/contributing/code-style-and-quality-common.html">
Common Coding Guide
</a>
</li>
<li class="list-group-item">
<a href="/contributing/code-style-and-quality-java.html">
Java Language Guide
</a>
</li>
<li class="list-group-item">
<a href="/contributing/code-style-and-quality-scala.html">
Scala Language Guide
</a>
</li>
<li class="list-group-item">
<a href="/contributing/code-style-and-quality-components.html">
Component Guides
</a>
</li>
<li class="list-group-item">
<a href="/contributing/code-style-and-quality-formatting.html">
Formatting Guide
</a>
</li>
</ul>
<hr />
<div class="page-toc">
<ul id="markdown-toc">
<li><a href="#java-language-features-and-libraries" id="markdown-toc-java-language-features-and-libraries">Java Language Features and Libraries</a> <ul>
<li><a href="#preconditions-and-log-statements" id="markdown-toc-preconditions-and-log-statements">Preconditions and Log Statements</a></li>
<li><a href="#generics" id="markdown-toc-generics">Generics</a></li>
<li><a href="#equals--hashcode" id="markdown-toc-equals--hashcode">equals() / hashCode()</a></li>
<li><a href="#java-serialization" id="markdown-toc-java-serialization">Java Serialization</a></li>
<li><a href="#java-reflection" id="markdown-toc-java-reflection">Java Reflection</a></li>
<li><a href="#collections" id="markdown-toc-collections">Collections</a></li>
<li><a href="#java-optional" id="markdown-toc-java-optional">Java Optional</a></li>
<li><a href="#lambdas" id="markdown-toc-lambdas">Lambdas</a></li>
<li><a href="#java-streams" id="markdown-toc-java-streams">Java Streams</a></li>
</ul>
</li>
</ul>
</div>
<h2 id="java-language-features-and-libraries">Java Language Features and Libraries</h2>
<h3 id="preconditions-and-log-statements">Preconditions and Log Statements</h3>
<ul>
<li>Never concatenate strings in the parameters
<ul>
<li><span style="text-decoration:underline;">Don’t:</span> <code>Preconditions.checkState(value &lt;= threshold, "value must be below " + threshold)</code></li>
<li><span style="text-decoration:underline;">Don’t:</span> <code>LOG.debug("value is " + value)</code></li>
<li><span style="text-decoration:underline;">Do:</span> <code>Preconditions.checkState(value &lt;= threshold, "value must be below %s", threshold)</code></li>
<li><span style="text-decoration:underline;">Do:</span> <code>LOG.debug("value is {}", value)</code></li>
</ul>
</li>
</ul>
<h3 id="generics">Generics</h3>
<ul>
<li><strong>No raw types:</strong> Do not use raw types, unless strictly necessary (sometimes necessary for signature matches, arrays).</li>
<li><strong>Suppress warnings for unchecked conversions:</strong> Add annotations to suppress warnings, if they cannot be avoided (such as “unchecked”, or “serial”). Otherwise warnings about generics flood the build and drown relevant warnings.</li>
</ul>
<h3 id="equals--hashcode">equals() / hashCode()</h3>
<ul>
<li><strong>equals() / hashCode() should be added when they are well defined only.</strong></li>
<li>They should <strong>not be added to enable a simpler assertion in tests</strong> when they are not well defined. Use hamcrest matchers in that case: <a href="https://github.com/junit-team/junit4/wiki/matchers-and-assertthat">https://github.com/junit-team/junit4/wiki/matchers-and-assertthat</a></li>
<li>A common indicator that the methods are not well defined is when they take a subset of the fields into account (other than fields that are purely auxiliary).</li>
<li>When the methods take mutable fields into account, you often have a design issue. The <code>equals()</code>/<code>hashCode()</code> methods suggest to use the type as a key, but the signatures suggest it is safe to keep mutating the type.</li>
</ul>
<h3 id="java-serialization">Java Serialization</h3>
<ul>
<li><strong>Do not use Java Serialization for anything !!!</strong></li>
<li><strong>Do not use Java Serialization for anything !!! !!!</strong></li>
<li><strong>Do not use Java Serialization for anything !!! !!! !!!</strong></li>
<li>Internal to Flink, Java serialization is used to transport messages and programs through RPC. This is the only case where we use Java serialization. Because of that, some classes need to be serializable (if they are transported via RPC).</li>
<li>
<p><strong>Serializable classes must define a Serial Version UID:</strong></p>
<p><code>private static final long serialVersionUID = 1L;</code></p>
</li>
<li><strong>The Serial Version UID for new classes should start at 1</strong> and should generally be bumped on every incompatible change to the class according to the Java serialization compatibility definition (i.e: changing the type of a field, or moving the position of a class in the class hierarchy).</li>
</ul>
<h3 id="java-reflection">Java Reflection</h3>
<p><strong>Avoid using Java’s Reflection API</strong></p>
<ul>
<li>Java’s Reflection API can be a very useful tool in certain cases but in all cases it is a hack and one should research for alternatives. The only cases where Flink should use reflection are
<ul>
<li>Dynamically loading implementations from another module (like webUI, additional serializers, pluggable query processors).</li>
<li>Extracting types inside the TypeExtractor class. This is fragile enough and should not be done outside the TypeExtractor class.</li>
<li>Some cases of cross-JDK version features, where we need to use reflection because we cannot assume a class/method to be present in all versions.</li>
</ul>
</li>
<li>If you need reflection for accessing methods or fields in tests, it usually indicates some deeper architectural issues, like wrong scoping, bad separation of concerns, or that there is no clean way to provide components / dependencies to the class that is tested</li>
</ul>
<h3 id="collections">Collections</h3>
<ul>
<li><strong>ArrayList and ArrayDeque are almost always superior to LinkedList</strong>, except when frequently insert and deleting in the middle of the list</li>
<li><strong>For Maps, avoid patterns that require multiple lookups</strong>
<ul>
<li><code>contains()</code> before <code>get()</code><code>get()</code> and check null</li>
<li><code>contains()</code> before <code>put()</code><code>putIfAbsent()</code> or <code>computeIfAbsent()</code></li>
<li>Iterating over keys, getting values → iterate over <code>entrySet()</code></li>
</ul>
</li>
<li><strong>Set the initial capacity for a collection only if there is a good proven reason</strong> for that, otherwise do not clutter the code. In case of <strong>Maps</strong> it can be even deluding because the Map’s load factor effectively reduces the capacity.</li>
</ul>
<h3 id="java-optional">Java Optional</h3>
<ul>
<li>Use <strong>@Nullable annotation where you do not use Optional</strong> for the nullable values.</li>
<li>If you can prove that <code>Optional</code> usage would lead to a <strong>performance degradation in critical code then fallback to @Nullable</strong>.</li>
<li>Always use <strong>Optional to return nullable values</strong> in the API/public methods except the case of a proven performance concern.</li>
<li><strong>Do not use Optional as a function argument</strong>, instead either overload the method or use the Builder pattern for the set of function arguments.
<ul>
<li>Note: an Optional argument can be allowed in a private helper method if you believe that it simplifies the code
(<a href="https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95">example</a>).</li>
</ul>
</li>
<li><strong>Do not use Optional for class fields</strong>.</li>
</ul>
<h3 id="lambdas">Lambdas</h3>
<ul>
<li>
<p>Prefer non-capturing lambdas (lambdas that do not contain references to the outer scope). Capturing lambdas need to create a new object instance for every call. Non-capturing lambdas can use the same instance for each invocation.</p>
<p><strong>don’t:</strong>
<code>
map.computeIfAbsent(key, x -&gt; key.toLowerCase())
</code></p>
<p><strong>do:</strong>
<code>
map.computeIfAbsent(key, k -&gt; k.toLowerCase());
</code></p>
</li>
<li>
<p>Consider method references instead of inline lambdas</p>
<p><strong>don’t</strong>:
<code>
map.computeIfAbsent(key, k-&gt; Loader.load(k));
</code></p>
<p><strong>do:</strong>
<code>
map.computeIfAbsent(key, Loader::load);
</code></p>
</li>
</ul>
<h3 id="java-streams">Java Streams</h3>
<ul>
<li>Avoid Java Streams in any performance critical code.</li>
<li>The main motivation to use Java Streams would be to improve code readability. As such, they can be a good match in parts of the code that are not data-intensive, but deal with coordination..</li>
<li>Even in the latter case, try to limit the scope to a method, or a few private methods within an internal class.</li>
</ul>
</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> &middot; <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>