blob: 6e048e17f3632089e023e63d6f3986ba66b60da4 [file] [log] [blame]
<!doctype html><html lang=en class=no-js><head><meta charset=utf-8><meta http-equiv=x-ua-compatible content="IE=edge"><meta name=viewport content="width=device-width,initial-scale=1"><title>PTransform Style Guide</title><meta name=description content="Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes."><link href="https://fonts.googleapis.com/css?family=Roboto:100,300,400,500,700" rel=stylesheet><link rel=preload href=/scss/main.min.d653ded46cd5f19a535cb20567fce9699849fe46f950d91ac6bf336db8ff8724.css as=style><link href=/scss/main.min.d653ded46cd5f19a535cb20567fce9699849fe46f950d91ac6bf336db8ff8724.css rel=stylesheet integrity><script src=https://code.jquery.com/jquery-2.2.4.min.js></script><style>.body__contained img{max-width:100%}</style><script type=text/javascript src=/js/bootstrap.min.2979f9a6e32fc42c3e7406339ee9fe76b31d1b52059776a02b4a7fa6a4fd280a.js defer></script>
<script type=text/javascript src=/js/language-switch-v2.min.121952b7980b920320ab229551857669209945e39b05ba2b433a565385ca44c6.js defer></script>
<script type=text/javascript src=/js/fix-menu.min.039174b67107465f2090a493f91e126f7aa797f29420f9edab8a54d9dd4b3d2d.js defer></script>
<script type=text/javascript src=/js/section-nav.min.1405fd5e70fab5f6c54037c269b1d137487d8f3d1b3009032525f6db3fbce991.js defer></script>
<script type=text/javascript src=/js/page-nav.min.af231204c9c52c5089d53a4c02739eacbb7f939e3be1c6ffcc212e0ac4dbf879.js defer></script>
<script type=text/javascript src=/js/expandable-list.min.75a4526624a3b8898fe7fb9e3428c205b581f8b38c7926922467aef17eac69f2.js defer></script>
<script type=text/javascript src=/js/copy-to-clipboard.min.364c06423d7e8993fc42bb4abc38c03195bc8386db26d18774ce775d08d5b18d.js defer></script>
<script type=text/javascript src=/js/calendar.min.336664054fa0f52b08bbd4e3c59b5cb6d63dcfb2b4d602839746516b0817446b.js defer></script>
<script type=text/javascript src=/js/fix-playground-nested-scroll.min.0283f1037cb1b9d5074c6eaf041292b524a8148a7cdb803d5ccd6d1fc4eb3253.js defer></script>
<script type=text/javascript src=/js/anchor-content-jump-fix.min.22d3240f81632e4c11179b9d2aaf37a40da9414333c43aa97344e8b21a7df0e4.js defer></script>
<link rel=alternate type=application/rss+xml title="Apache Beam" href=/feed.xml><link rel=canonical href=/contribute/ptransform-style-guide/ data-proofer-ignore><link rel="shortcut icon" type=image/x-icon href=/images/favicon.ico><link rel=stylesheet href=https://use.fontawesome.com/releases/v5.4.1/css/all.css integrity=sha384-5sAR7xN1Nv6T6+dT2mhtzEpVJvfS3NScPQTrOxhwjIuvcA67KV2R5Jz6kr4abQsz crossorigin=anonymous><link rel=stylesheet href=https://unpkg.com/swiper@8/swiper-bundle.min.css><script async src=https://platform.twitter.com/widgets.js></script>
<script>(function(e,t,n,s,o,i,a){e.GoogleAnalyticsObject=o,e[o]=e[o]||function(){(e[o].q=e[o].q||[]).push(arguments)},e[o].l=1*new Date,i=t.createElement(n),a=t.getElementsByTagName(n)[0],i.async=1,i.src=s,a.parentNode.insertBefore(i,a)})(window,document,"script","//www.google-analytics.com/analytics.js","ga"),ga("create","UA-73650088-1","auto"),ga("send","pageview")</script><script>(function(e,t,n,s,o,i){e.hj=e.hj||function(){(e.hj.q=e.hj.q||[]).push(arguments)},e._hjSettings={hjid:2182187,hjsv:6},o=t.getElementsByTagName("head")[0],i=t.createElement("script"),i.async=1,i.src=n+e._hjSettings.hjid+s+e._hjSettings.hjsv,o.appendChild(i)})(window,document,"https://static.hotjar.com/c/hotjar-",".js?sv=")</script></head><body class=body data-spy=scroll data-target=.page-nav data-offset=0><nav class="navigation-bar-mobile header navbar navbar-fixed-top"><div class=navbar-header><a href=/ class=navbar-brand><img alt=Brand style=height:46px;width:43px src=/images/beam_logo_navbar_mobile.png></a>
<a class=navbar-link href=/get-started/>Get Started</a>
<a class=navbar-link href=/documentation/>Documentation</a>
<button type=button class="navbar-toggle menu-open" aria-expanded=false aria-controls=navbar onclick=openMenu()>
<span class=sr-only>Toggle navigation</span>
<span class=icon-bar></span>
<span class=icon-bar></span>
<span class=icon-bar></span></button></div><div class="navbar-mask closed"></div><div id=navbar class="navbar-container closed"><button type=button class=navbar-toggle aria-expanded=false aria-controls=navbar id=closeMenu>
<span class=sr-only>Toggle navigation</span>
<span class=icon-bar></span>
<span class=icon-bar></span>
<span class=icon-bar></span></button><ul class="nav navbar-nav"><li><div class=searchBar-mobile><script>(function(){var t,n="012923275103528129024:4emlchv9wzi",e=document.createElement("script");e.type="text/javascript",e.async=!0,e.src="https://cse.google.com/cse.js?cx="+n,t=document.getElementsByTagName("script")[0],t.parentNode.insertBefore(e,t)})()</script><gcse:search></gcse:search></div></li><li><a class=navbar-link href=/about>About</a></li><li><a class=navbar-link href=/get-started/>Get Started</a></li><li><span class=navbar-link>Documentation</span><ul><li><a href=/documentation/>General</a></li><li><a href=/documentation/sdks/java/>Languages</a></li><li><a href=/documentation/runners/capability-matrix/>Runners</a></li><li><a href=/documentation/io/connectors/>I/O Connectors</a></li></ul></li><li><a class=navbar-link href=/roadmap/>Roadmap</a></li><li><a class=navbar-link href=/community/>Community</a></li><li><a class=navbar-link href=/contribute/>Contribute</a></li><li><a class=navbar-link href=/blog/>Blog</a></li><li><a class=navbar-link href=/case-studies/>Case Studies</a></li></ul><ul class="nav navbar-nav navbar-right"><li><a href=https://github.com/apache/beam/edit/master/website/www/site/content/en/contribute/ptransform-style-guide.md data-proofer-ignore><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M4.543 20h4l10.5-10.5c.53-.53.828-1.25.828-2s-.298-1.47-.828-2-1.25-.828-2-.828-1.47.298-2 .828L4.543 16v4zm9.5-13.5 4 4"/></svg></a></li><li class=dropdown><a href=# class=dropdown-toggle id=apache-dropdown data-toggle=dropdown role=button aria-haspopup=true aria-expanded=false><img src=https://www.apache.org/foundation/press/kit/feather_small.png alt="Apache Logo" style=height:20px>
&nbsp;Apache
<span class=arrow-icon><svg xmlns="http://www.w3.org/2000/svg" width="20" height="20" fill="none" viewBox="0 0 20 20"><circle cx="10" cy="10" r="10" fill="#ff6d00"/><path stroke="#fff" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M8.535 5.28l4.573 4.818-4.573 4.403"/></svg></span></a><ul class="dropdown-menu dropdown-menu-right"><li><a target=_blank href=https://www.apache.org/>ASF Homepage</a></li><li><a target=_blank href=https://www.apache.org/licenses/>License</a></li><li><a target=_blank href=https://www.apache.org/security/>Security</a></li><li><a target=_blank href=https://www.apache.org/foundation/thanks.html>Thanks</a></li><li><a target=_blank href=https://www.apache.org/foundation/sponsorship.html>Sponsorship</a></li><li><a target=_blank href=https://www.apache.org/foundation/policies/conduct>Code of Conduct</a></li></ul></li></ul></div></nav><nav class=navigation-bar-desktop><a href=/ class=navbar-logo><img src=/images/beam_logo_navbar.png alt="Beam Logo"></a><div class=navbar-bar-left><div class=navbar-links><a class=navbar-link href=/about>About</a>
<a class=navbar-link href=/get-started/>Get Started</a><li class="dropdown navbar-dropdown navbar-dropdown-documentation"><a href=# class="dropdown-toggle navbar-link" role=button aria-haspopup=true aria-expanded=false>Documentation
<span><svg xmlns="http://www.w3.org/2000/svg" width="12" height="11" fill="none" viewBox="0 0 12 11"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M10.666 4.535 5.847 9.108 1.444 4.535"/></svg></span></a><ul class=dropdown-menu><li><a class=navbar-dropdown-menu-link href=/documentation/>General</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/sdks/java/>Languages</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/runners/capability-matrix/>Runners</a></li><li><a class=navbar-dropdown-menu-link href=/documentation/io/connectors/>I/O Connectors</a></li></ul></li><a class=navbar-link href=/roadmap/>Roadmap</a>
<a class=navbar-link href=/community/>Community</a>
<a class=navbar-link href=/contribute/>Contribute</a>
<a class=navbar-link href=/blog/>Blog</a>
<a class=navbar-link href=/case-studies/>Case Studies</a></div><div id=iconsBar><a type=button onclick=showSearch()><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M10.191 17c3.866.0 7-3.134 7-7s-3.134-7-7-7-7 3.134-7 7 3.134 7 7 7zm11 4-6-6"/></svg></a><a target=_blank href=https://github.com/apache/beam/edit/master/website/www/site/content/en/contribute/ptransform-style-guide.md data-proofer-ignore><svg xmlns="http://www.w3.org/2000/svg" width="25" height="24" fill="none" viewBox="0 0 25 24"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M4.543 20h4l10.5-10.5c.53-.53.828-1.25.828-2s-.298-1.47-.828-2-1.25-.828-2-.828-1.47.298-2 .828L4.543 16v4zm9.5-13.5 4 4"/></svg></a><li class="dropdown navbar-dropdown navbar-dropdown-apache"><a href=# class=dropdown-toggle role=button aria-haspopup=true aria-expanded=false><img src=https://www.apache.org/foundation/press/kit/feather_small.png alt="Apache Logo" style=height:20px>
&nbsp;Apache
<span class=arrow-icon><svg xmlns="http://www.w3.org/2000/svg" width="20" height="20" fill="none" viewBox="0 0 20 20"><circle cx="10" cy="10" r="10" fill="#ff6d00"/><path stroke="#fff" stroke-linecap="round" stroke-linejoin="round" stroke-width="2" d="M8.535 5.28l4.573 4.818-4.573 4.403"/></svg></span></a><ul class=dropdown-menu><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/>ASF Homepage</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/licenses/>License</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/security/>Security</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/thanks.html>Thanks</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/sponsorship.html>Sponsorship</a></li><li><a class=navbar-dropdown-menu-link target=_blank href=https://www.apache.org/foundation/policies/conduct>Code of Conduct</a></li></ul></li></div><div class="searchBar disappear"><script>(function(){var t,n="012923275103528129024:4emlchv9wzi",e=document.createElement("script");e.type="text/javascript",e.async=!0,e.src="https://cse.google.com/cse.js?cx="+n,t=document.getElementsByTagName("script")[0],t.parentNode.insertBefore(e,t)})()</script><gcse:search></gcse:search>
<a type=button onclick=endSearch()><svg xmlns="http://www.w3.org/2000/svg" width="25" height="25" fill="none" viewBox="0 0 25 25"><path stroke="#ff6d00" stroke-linecap="round" stroke-linejoin="round" stroke-width="2.75" d="M21.122 20.827 4.727 4.432M21.122 4.43 4.727 20.827"/></svg></a></div></div></nav><div class=header-push></div><div class="top-banners swiper"><div class=swiper-wrapper><div class=swiper-slide><a href=https://tour.beam.apache.org><img class=banner-img-desktop src=/images/banners/tour-of-beam/tour-of-beam-desktop.png alt="Start Tour of Beam">
<img class=banner-img-mobile src=/images/banners/tour-of-beam/tour-of-beam-mobile.png alt="Start Tour of Beam"></a></div><div class=swiper-slide><a href=https://beam.apache.org/documentation/ml/overview/><img class=banner-img-desktop src=/images/banners/machine-learning/machine-learning-desktop.jpg alt="Machine Learning">
<img class=banner-img-mobile src=/images/banners/machine-learning/machine-learning-mobile.jpg alt="Machine Learning"></a></div></div><div class=swiper-pagination></div><div class=swiper-button-prev></div><div class=swiper-button-next></div></div><script src=/js/swiper-bundle.min.min.e0e8f81b0b15728d35ff73c07f42ddbb17a108d6f23df4953cb3e60df7ade675.js></script>
<script src=/js/sliders/top-banners.min.afa7d0a19acf7a3b28ca369490b3d401a619562a2a4c9612577be2f66a4b9855.js></script>
<script>function showSearch(){addPlaceholder();var e,t=document.querySelector(".searchBar");t.classList.remove("disappear"),e=document.querySelector("#iconsBar"),e.classList.add("disappear")}function addPlaceholder(){$("input:text").attr("placeholder","What are you looking for?")}function endSearch(){var e,t=document.querySelector(".searchBar");t.classList.add("disappear"),e=document.querySelector("#iconsBar"),e.classList.remove("disappear")}function blockScroll(){$("body").toggleClass("fixedPosition")}function openMenu(){addPlaceholder(),blockScroll()}</script><div class="clearfix container-main-content"><div class="section-nav closed" data-offset-top=90 data-offset-bottom=500><span class="section-nav-back glyphicon glyphicon-menu-left"></span><nav><ul class=section-nav-list data-section-nav><li><span class=section-nav-list-main-title>Contribute</span></li><li><a href=https://github.com/apache/beam/blob/master/CONTRIBUTING.md>Code contribution guide</a></li><li><a href=/contribute/get-help/>Get help</a></li><li><a href=/contribute/attributes/>Attributes of a Beam community member</a></li><li><span class=section-nav-list-title>Technical Docs</span><ul class=section-nav-list><li><a href=https://cwiki.apache.org/confluence/display/BEAM/Contribution+Testing+Guide>Testing guide</a></li><li><a href=/contribute/precommit-triage-guide/>Pre-commit slowness triage</a></li><li><a href=/contribute/ptransform-style-guide/>PTransform style guide</a></li><li><a href=/contribute/runner-guide/>Runner authoring guide</a></li><li><a href=/contribute/dependencies/>Dependencies guide</a></li></ul></li><li><span class=section-nav-list-title>Policies</span><ul class=section-nav-list><li><a href=/contribute/issue-priorities/>Issue priorities</a></li><li><a href=/contribute/precommit-policies/>Pre-commit test policies</a></li><li><a href=/contribute/postcommits-policies/>Post-commit test policies</a></li><li><a href=/contribute/release-blockers/>Release blockers</a></li></ul></li><li><span class=section-nav-list-title>Committers</span><ul class=section-nav-list><li><a href=/contribute/become-a-committer/>Become a committer</a></li></ul></li></ul></nav></div><nav class="page-nav clearfix" data-offset-top=90 data-offset-bottom=500><nav id=TableOfContents><ul><li><a href=#language-neutral-considerations>Language-neutral considerations</a><ul><li><a href=#consistency>Consistency</a></li><li><a href=#exposing-a-ptransform-vs-something-else>Exposing a PTransform vs. something else</a></li><li><a href=#naming>Naming</a></li><li><a href=#configuration>Configuration</a><ul><li><a href=#what-goes-into-configuration-vs-input-collection>What goes into configuration vs. input collection</a></li><li><a href=#what-parameters-to-expose>What parameters to expose</a></li></ul></li><li><a href=#error-handling>Error handling</a><ul><li><a href=#transform-configuration-errors>Transform configuration errors</a></li><li><a href=#runtime-errors-and-data-consistency>Runtime errors and data consistency</a></li></ul></li><li><a href=#performance>Performance</a></li><li><a href=#documentation>Documentation</a></li><li><a href=#logging>Logging</a></li><li><a href=#testing>Testing</a><ul><li><a href=#testing-the-transforms-run-time-behavior>Testing the transform&rsquo;s run-time behavior</a></li><li><a href=#testing-transform-construction-and-validation>Testing transform construction and validation</a></li></ul></li><li><a href=#compatibility>Compatibility</a></li></ul></li><li><a href=#java-specific-considerations>Java specific considerations</a><ul><li><a href=#api>API</a><ul><li><a href=#choosing-types-of-input-and-output-pcollections>Choosing types of input and output PCollection&rsquo;s</a></li><li><a href=#transforms-with-multiple-output-collections>Transforms with multiple output collections</a></li><li><a href=#fluent-builders-for-configuration>Fluent builders for configuration</a></li><li><a href=#transforms-with-type-parameters>Transforms with type parameters</a></li><li><a href=#injecting-user-specified-behavior>Injecting user-specified behavior</a></li><li><a href=#packaging-a-family-of-transforms>Packaging a family of transforms</a></li></ul></li><li><a href=#behavior>Behavior</a><ul><li><a href=#immutability>Immutability</a></li><li><a href=#serialization>Serialization</a></li><li><a href=#validation>Validation</a></li><li><a href=#coders>Coders</a></li></ul></li></ul></li></ul></nav></nav><div class="body__contained body__section-nav"><h1 id=ptransform-style-guide>PTransform Style Guide</h1><p><em>A style guide for writers of new reusable PTransforms.</em></p><nav id=TableOfContents><ul><li><a href=#language-neutral-considerations>Language-neutral considerations</a><ul><li><a href=#consistency>Consistency</a></li><li><a href=#exposing-a-ptransform-vs-something-else>Exposing a PTransform vs. something else</a></li><li><a href=#naming>Naming</a></li><li><a href=#configuration>Configuration</a><ul><li><a href=#what-goes-into-configuration-vs-input-collection>What goes into configuration vs. input collection</a></li><li><a href=#what-parameters-to-expose>What parameters to expose</a></li></ul></li><li><a href=#error-handling>Error handling</a><ul><li><a href=#transform-configuration-errors>Transform configuration errors</a></li><li><a href=#runtime-errors-and-data-consistency>Runtime errors and data consistency</a></li></ul></li><li><a href=#performance>Performance</a></li><li><a href=#documentation>Documentation</a></li><li><a href=#logging>Logging</a></li><li><a href=#testing>Testing</a><ul><li><a href=#testing-the-transforms-run-time-behavior>Testing the transform&rsquo;s run-time behavior</a></li><li><a href=#testing-transform-construction-and-validation>Testing transform construction and validation</a></li></ul></li><li><a href=#compatibility>Compatibility</a></li></ul></li><li><a href=#java-specific-considerations>Java specific considerations</a><ul><li><a href=#api>API</a><ul><li><a href=#choosing-types-of-input-and-output-pcollections>Choosing types of input and output PCollection&rsquo;s</a></li><li><a href=#transforms-with-multiple-output-collections>Transforms with multiple output collections</a></li><li><a href=#fluent-builders-for-configuration>Fluent builders for configuration</a></li><li><a href=#transforms-with-type-parameters>Transforms with type parameters</a></li><li><a href=#injecting-user-specified-behavior>Injecting user-specified behavior</a></li><li><a href=#packaging-a-family-of-transforms>Packaging a family of transforms</a></li></ul></li><li><a href=#behavior>Behavior</a><ul><li><a href=#immutability>Immutability</a></li><li><a href=#serialization>Serialization</a></li><li><a href=#validation>Validation</a></li><li><a href=#coders>Coders</a></li></ul></li></ul></li></ul></nav><h2 id=language-neutral-considerations>Language-neutral considerations</h2><h3 id=consistency>Consistency</h3><p>Be consistent with prior art:</p><ul><li>Please read the <a href=/contribute/>contribution guide</a>.</li><li>If there is already a similar transform in some SDK, make the API of your transform similar, so that users&rsquo; experience with one of them will transfer to the other. This applies to transforms in the same-language SDK and different-language SDKs.
<em>Exception:</em> pre-existing transforms that clearly violate the current style guide for the sole reason that they were developed before this guide was ratified. In this case, the style guide takes priority over consistency with the existing transform.</li><li>When there is no existing similar transform, stay within what is idiomatic within your language of choice (e.g. Java or Python).</li><li>Please use this <a href=https://s.apache.org/ptransform-design-doc>design doc template</a> if possible, when proposing new transforms.</li></ul><h3 id=exposing-a-ptransform-vs-something-else>Exposing a PTransform vs. something else</h3><p>So you want to develop a library that people will use in their Beam pipelines - a connector to a third-party system, a machine learning algorithm, etc. How should you expose it?</p><p>Do:</p><ul><li>Expose every major data-parallel task accomplished by your library as a composite <code>PTransform</code>. This allows the structure of the transform to evolve transparently to the code that uses it: e.g. something that started as a <code>ParDo</code> can become a more complex transform over time.</li><li>Expose large, non-trivial, reusable sequential bits of the transform&rsquo;s code, which others might want to reuse in ways you haven&rsquo;t anticipated, as a regular function or class library. The transform should simply wire this logic together. As a side benefit, you can unit-test those functions and classes independently.
<em>Example:</em> when developing a transform that parses files in a custom data format, expose the format parser as a library; likewise for a transform that implements a complex machine learning algorithm, etc.</li><li>In some cases, this may include Beam-specific classes, such as <code>CombineFn</code>, or nontrivial <code>DoFn</code>s (those that are more than just a single <code>@ProcessElement</code> method).
As a rule of thumb: expose these if you anticipate that the full packaged <code>PTransform</code> may be insufficient for a user&rsquo;s needs and the user may want to reuse the lower-level primitive.</li></ul><p>Do not:</p><ul><li>Do not expose the exact way the transform is internally structured. E.g.: the public API surface of your library <em>usually</em> (with exception of the last bullet above) should not expose <code>DoFn</code>, concrete <code>Source</code> or <code>Sink</code> classes, etc., in order to avoid presenting users with a confusing choice between applying the <code>PTransform</code> or using the <code>DoFn</code>/<code>Source</code>/<code>Sink</code>.</li></ul><h3 id=naming>Naming</h3><p>Do:</p><ul><li>Respect language-specific naming conventions, e.g. name classes in <code>PascalCase</code> in Java and Python, functions in <code>camelCase</code> in Java but <code>snake_case</code> in Python, etc.</li><li>Name factory functions so that either the function name is a verb, or referring to the transform reads like a verb: e.g. <code>MongoDbIO.read()</code>, <code>Flatten.iterables()</code>.</li><li>In typed languages, name <code>PTransform</code> classes also like verbs (e.g.: <code>MongoDbIO.Read</code>, <code>Flatten.Iterables</code>).</li><li>Name families of transforms for interacting with a storage system using the word &ldquo;IO&rdquo;: <code>MongoDbIO</code>, <code>JdbcIO</code>.</li></ul><p>Do not:</p><ul><li>Do not use words <code>transform</code>, <code>source</code>, <code>sink</code>, <code>reader</code>, <code>writer</code>, <code>bound</code>, <code>unbound</code> in <code>PTransform</code> class names (note: <code>bounded</code> and <code>unbounded</code> are fine when referring to whether a <code>PCollection</code> is bounded or unbounded): these words are redundant, confusing, obsolete, or name an existing different concept in the SDK.</li></ul><h3 id=configuration>Configuration</h3><h4 id=what-goes-into-configuration-vs-input-collection>What goes into configuration vs. input collection</h4><ul><li><strong>Into input <code>PCollection</code>:</strong> anything of which there may be a very large number of instances (if there can be >1000 of it, it should be in a <code>PCollection</code>), or which is potentially not known at pipeline construction time.
E.g.: records to be processed or written to a third-party system; filenames to be read.
Exception: sometimes Beam APIs require things to be known at pipeline construction time - e.g. the <code>Bounded</code>/<code>UnboundedSource</code> API. If you absolutely have to use such an API, its input can of course go only into transform configuration.</li><li><strong>Into transform configuration:</strong> what is constant throughout the transform (including <code>ValueProvider</code>s) and does not depend on the contents of the transform&rsquo;s input <code>PCollection</code>s.
E.g.: a database query or connection string; credentials; a user-specified callback; a tuning parameter.
One advantage of putting a parameter into transform configuration is, it can be validated at pipeline construction time.</li></ul><h4 id=what-parameters-to-expose>What parameters to expose</h4><p>Do:</p><ul><li><strong>Expose</strong> parameters that are necessary to compute the output.</li></ul><p>Do not:</p><ul><li><strong>Do not expose</strong> tuning knobs, such as batch sizes, connection pool sizes, unless it&rsquo;s impossible to automatically supply or compute a good-enough value (i.e., unless you can imagine a reasonable person reporting a bug about the absence of this knob).</li><li>When developing a connector to a library that has many parameters, <strong>do not mirror each parameter</strong> of the underlying library - if necessary, reuse the underlying library&rsquo;s configuration class and let user supply a whole instance. Example: <code>JdbcIO</code>.
<em>Exception 1:</em> if some parameters of the underlying library interact with Beam semantics non-trivially, then expose them. E.g. when developing a connector to a pub/sub system that has a &ldquo;delivery guarantee&rdquo; parameter for publishers, expose the parameter but prohibit values incompatible with the Beam model (at-most-once and exactly-once).
<em>Exception 2:</em> if the underlying library&rsquo;s configuration class is cumbersome to use - e.g. does not declare a stable API, exposes problematic transitive dependencies, or does not obey <a href=https://semver.org/>semantic versioning</a> - in this case, it is better to wrap it and expose a cleaner and more stable API to users of the transform.</li></ul><h3 id=error-handling>Error handling</h3><h4 id=transform-configuration-errors>Transform configuration errors</h4><p>Detect errors early. Errors can be detected at the following stages:</p><ul><li>(in a compiled language) compilation of the source code of a user&rsquo;s pipeline</li><li>constructing or setting up the transform</li><li>applying the transform in a pipeline</li><li>running the pipeline</li></ul><p>For example:</p><ul><li>In a typed language, take advantage of compile-time error checking by making the API of the transform strongly-typed:<ul><li><strong>Strongly-typed configuration:</strong> e.g. in Java, a parameter that is a URL should use the <code>URL</code> class, rather than the <code>String</code> class.</li><li><strong>Strongly-typed input and output:</strong> e.g. a transform that writes to Mongo DB should take a <code>PCollection&lt;Document></code> rather than <code>PCollection&lt;String></code> (assuming it is possible to provide a <code>Coder</code> for <code>Document</code>).</li></ul></li><li>Detect invalid values of individual parameters in setter methods.</li><li>Detect invalid combinations of parameters in the transform&rsquo;s validate method.</li></ul><h4 id=runtime-errors-and-data-consistency>Runtime errors and data consistency</h4><p>Favor data consistency above everything else. Do not mask data loss or corruption. If data loss can&rsquo;t be prevented, fail.</p><p>Do:</p><ul><li>In a <code>DoFn</code>, retry transient failures if the operation is likely to succeed on retry. Perform such retries at the narrowest scope possible in order to minimize the amount of retried work (i.e. ideally at the level of the RPC library itself, or at the level of directly sending the failing RPC to a third-party system). Otherwise, let the runner retry work at the appropriate level of granularity for you (different runners may have different retry behavior, but most of them do <em>some</em> retrying).</li><li>If the transform has side effects, strive to make them idempotent (i.e. safe to apply multiple times). Due to retries, the side effects may be executed multiple times, possibly in parallel.</li><li>If the transform can have unprocessable (permanently failing) records and you want the pipeline to proceed despite that:<ul><li>If bad records are safe to ignore, count the bad records in a metric. Make sure the transform&rsquo;s documentation mentions this aggregator. Beware that there is no programmatic access to reading the aggregator value from inside the pipeline during execution.</li><li>If bad records may need manual inspection by the user, emit them into an output that contains only those records.</li><li>Alternatively take a (default zero) threshold above which element failures become bundle failures (structure the transform to count the total number of elements and of failed elements, compare them and fail if failures are above the threshold).</li></ul></li><li>If the user requests a higher data consistency guarantee than you&rsquo;re able to provide, fail. E.g.: if a user requests QoS 2 (exactly-once delivery) from an MQTT connector, the connector should fail since Beam runners may retry writing to the connector and hence exactly-once delivery can&rsquo;t be done.</li></ul><p>Do not:</p><ul><li>If you can&rsquo;t handle a failure, don&rsquo;t even catch it.
*Exception: *It may be valuable to catch the error, log a message, and rethrow it, if you&rsquo;re able to provide valuable context that the original error doesn&rsquo;t have.</li><li>Never, ever, ever do this:
<code>catch(...) { log an error; return null or false or otherwise ignore; }</code>
<strong>Rule of thumb: if a bundle didn&rsquo;t fail, its output must be correct and complete.</strong>
For a user, a transform that logged an error but succeeded is silent data loss.</li></ul><h3 id=performance>Performance</h3><p>Many runners optimize chains of <code>ParDo</code>s in ways that improve performance if the <code>ParDo</code>s emit a small to moderate number of elements per input element, or have relatively cheap per-element processing (e.g. Dataflow&rsquo;s &ldquo;fusion&rdquo;), but limit parallelization if these assumptions are violated. In that case you may need a &ldquo;fusion break&rdquo; (<code>Reshuffle.of()</code>) to improve the parallelizability of processing the output <code>PCollection</code> of the <code>ParDo</code>.</p><ul><li>If the transform includes a <code>ParDo</code> that outputs a potentially large number of elements per input element, apply a fusion break after this <code>ParDo</code> to make sure downstream transforms can process its output in parallel.</li><li>If the transform includes a <code>ParDo</code> that takes a very long time to process an element, insert a fusion break before this <code>ParDo</code> to make sure all or most elements can be processed in parallel regardless of how its input <code>PCollection</code> was produced.</li></ul><h3 id=documentation>Documentation</h3><p>Document how to configure the transform (give code examples), and what guarantees it expects about its input or provides about its output, accounting for the Beam model. E.g.:</p><ul><li>Are the input and output collections of this transform bounded or unbounded, or can it work with either?</li><li>If the transform writes data to a third-party system, does it guarantee that data will be written at least once? at most once? exactly once? (how does it achieve exactly-once in case the runner executes a bundle multiple times due to retries or speculative execution a.k.a. backups?)</li><li>If the transform reads data from a third-party system, what&rsquo;s the maximum potential degree of parallelism of the read? E.g., if the transform reads data sequentially (e.g. executes a single SQL query), documentation should mention that.</li><li>If the transform is querying an external system during processing (e.g. joining a <code>PCollection</code> with information from an external key-value store), what are the guarantees on freshness of queried data: e.g. is it all loaded at the beginning of the transform, or queried per-element (in that case, what if data for a single element changes while the transform runs)?</li><li>If there&rsquo;s a non-trivial relationship between arrival of items in the input <code>PCollection</code> and emitting output into the output <code>PCollection</code>, what is this relationship? (e.g. if the transform internally does windowing, triggering, grouping, or uses the state or timers API)</li></ul><h3 id=logging>Logging</h3><p>Anticipate abnormal situations that a user of the transform may run into. Log information that they would have found sufficient for debugging, but limit the volume of logging. Here is some advice that applies to all programs, but is especially important when data volume is massive and execution is distributed.</p><p>Do:</p><ul><li>When handling an error from a third-party system, log the full error with any error details the third-party system provides about it, and include any additional context the transform knows. This enables the user to take action based on the information provided in the message. When handling an exception and rethrowing your own exception, wrap the original exception in it (some languages offer more advanced facilities, e.g. Java&rsquo;s &ldquo;suppressed exceptions&rdquo;). Never silently drop available information about an error.</li><li>When performing a rare (not per-element) and slow operation (e.g. expanding a large file-pattern, or initiating an import/export job), log when the operation begins and ends. If the operation has an identifier, log the identifier, so the user can look up the operation for later debugging.</li><li>When computing something low-volume that is critically important for correctness or performance of further processing, log the input and output, so a user in the process of debugging can sanity-check them or reproduce an abnormal result manually.
E.g. when expanding a filepattern into files, log what the filepattern was and how many parts it was split into; when executing a query, log the query and log how many results it produced.</li></ul><p>Do not:</p><ul><li>Do not log at <code>INFO</code> per element or per bundle. <code>DEBUG</code>/<code>TRACE</code> may be okay because these levels are disabled by default.</li><li>Avoid logging data payloads that may contain sensitive information, or sanitize them before logging (e.g. user data, credentials, etc).</li></ul><h3 id=testing>Testing</h3><p>Data processing is tricky, full of corner cases, and difficult to debug, because pipelines take a long time to run, it&rsquo;s hard to check if the output is correct, you can&rsquo;t attach a debugger, and you often can&rsquo;t log as much as you wish to, due to high volume of data. Because of that, testing is particularly important.</p><h4 id=testing-the-transforms-run-time-behavior>Testing the transform&rsquo;s run-time behavior</h4><ul><li>Unit-test the overall semantics of the transform using <code>TestPipeline</code> and <code>PAssert</code>. Start with testing against the direct runner. Assertions on <code>PCollection</code> contents should be strict: e.g. when a read from a database is expected to read the numbers 1 through 10, assert not just that there are 10 elements in the output <code>PCollection</code>, or that each element is in the range [1, 10] - but assert that each number 1 through 10 appears exactly once.</li><li>Identify non-trivial sequential logic in the transform that is prone to corner cases which are difficult to reliably simulate using a <code>TestPipeline</code>, extract this logic into unit-testable functions, and unit-test them. Common corner cases are:<ul><li><code>DoFn</code>s processing empty bundles</li><li><code>DoFn</code>s processing extremely large bundles (contents doesn&rsquo;t fit in memory, including &ldquo;hot keys&rdquo; with a very large number of values)</li><li>Third-party APIs failing</li><li>Third-party APIs providing wildly inaccurate information</li><li>Leaks of <code>Closeable</code>/<code>AutoCloseable</code> resources in failure cases</li><li>Common corner cases when developing sources: complicated arithmetic in <code>BoundedSource.split</code> (e.g. splitting key or offset ranges), iteration over empty data sources or composite data sources that have some empty components.</li></ul></li><li>Mock out the interactions with third-party systems, or better, use <a href=https://martinfowler.com/articles/mocksArentStubs.html>&ldquo;fake&rdquo;</a> implementations when available. Make sure that the mocked-out interactions are representative of all interesting cases of the actual behavior of these systems.</li><li>To unit test <code>DoFn</code>s, <code>CombineFn</code>s, and <code>BoundedSource</code>s, consider using <code>DoFnTester</code>, <code>CombineFnTester</code>, and <code>SourceTestUtils</code> respectively which can exercise the code in non-trivial ways to flesh out potential bugs.</li><li>For transforms that work over unbounded collections, test their behavior in the presence of late or out-of-order data using <code>TestStream</code>.</li><li>Tests must pass 100% of the time, including in hostile, CPU- or network-constrained environments (continuous integration servers). Never put timing-dependent code (e.g. sleeps) into tests. Experience shows that no reasonable amount of sleeping is enough - code can be suspended for more than several seconds.</li><li>For detailed instructions on test code organization, see the <a href=https://cwiki.apache.org/confluence/display/BEAM/Contribution+Testing+Guide>Beam Testing Guide</a>.</li></ul><h4 id=testing-transform-construction-and-validation>Testing transform construction and validation</h4><p>The code for constructing and validating a transform is usually trivial and mostly boilerplate. However, minor mistakes or typos in it can have serious consequences (e.g. ignoring a property that the user has set), so it needs to be tested as well. Yet, an excessive amount of trivial tests can be hard to maintain and give a false impression that the transform is well-tested.</p><p>Do:</p><ul><li>Test non-trivial validation code, where missing/incorrect/uninformative validation may lead to serious problems: data loss, counter-intuitive behavior, value of a property being silently ignored, or other hard-to-debug errors. Create 1 test per non-trivial class of validation error. Some examples of validation that should be tested:<ul><li>If properties <code>withFoo()</code> and <code>withBar()</code> cannot both be specified at the same time, test that a transform specifying both of them is rejected, rather than one of the properties being silently ignored at runtime.</li><li>If the transform is known to behave incorrectly or counter-intuitively for a particular configuration, test that this configuration is rejected, rather than producing wrong results at runtime. For example, a transform might work properly only for bounded collections, or only for globally-windowed collections. Or, suppose a streaming system supports several levels of &ldquo;quality of service&rdquo;, one of which is &ldquo;exactly once delivery&rdquo;. However, a transform that writes to this system might be unable to provide exactly-once due to retries in case of failures. In that case, test that the transform disallows specifying exactly-once QoS, rather than failing to provide the expected end-to-end semantics at runtime.</li></ul></li><li>Test that each <code>withFoo()</code> method (including each overload) has effect (is not ignored), using <code>TestPipeline</code> and <code>PAssert</code> to create tests where the expected test results depend on the value of <code>withFoo()</code>.</li></ul><p>Do not:</p><ul><li>Do not test successful validation (e.g. &ldquo;validation does not fail when the transform is configured correctly&rdquo;)</li><li>Do not test trivial validation errors (e.g. &ldquo;validation fails when a property is unset/null/empty/negative/&mldr;&rdquo;)</li></ul><h3 id=compatibility>Compatibility</h3><p>Do:</p><ul><li>Generally, follow the rules of <a href=https://semver.org/>semantic versioning</a>.</li><li>If the API of the transform is not yet stable, annotate it as <code>@Experimental</code> (Java) or <code>@experimental</code> (<a href=https://beam.apache.org/releases/pydoc/2.55.1/apache_beam.utils.annotations.html>Python</a>).</li><li>If the API deprecated, annotate it as <code>@Deprecated</code> (Java) or <code>@deprecated</code> (<a href=https://beam.apache.org/releases/pydoc/2.55.1/apache_beam.utils.annotations.html>Python</a>).</li><li>Pay attention to the stability and versioning of third-party classes exposed by the transform&rsquo;s API: if they are unstable or improperly versioned (do not obey <a href=https://semver.org/>semantic versioning</a>), it is better to wrap them in your own classes.</li></ul><p>Do not:</p><ul><li>Do not silently change the behavior of the transform, in a way where code will keep compiling but will do something different than the previously documented behavior (e.g. produce different output or expect different input, of course unless the previous output was incorrect).
Strive to make such incompatible behavior changes cause a compile error (e.g. it&rsquo;s better to introduce a new transform for a new behavior and deprecate and then delete the old one (in a new major version), than to silently change the behavior of an existing transform), or at least a runtime error.</li><li>If the behavior of the transform stays the same and you&rsquo;re merely changing implementation or API - do not change API of the transform in a way that will make a user&rsquo;s code stop compiling.</li></ul><h2 id=java-specific-considerations>Java specific considerations</h2><p>Good examples for most of the practices below are <code>JdbcIO</code> and <code>MongoDbIO</code>.</p><h3 id=api>API</h3><h4 id=choosing-types-of-input-and-output-pcollections>Choosing types of input and output PCollection&rsquo;s</h4><p>Whenever possible, use types specific to the nature of the transform. People can wrap it with conversion <code>DoFn</code>s from their own types if necessary. E.g. a Datastore connector should use the Datastore <code>Entity</code> type, a MongoDb connector should use Mongo <code>Document</code> type, not a String representation of the JSON.</p><p>Sometimes that&rsquo;s not possible (e.g. JDBC does not provide a Beam-compatible (encodable with a Coder) &ldquo;JDBC record&rdquo; datatype) - then let the user provide a function for converting between the transform-specific type and a Beam-compatible type (e.g. see <code>JdbcIO</code> and <code>MongoDbGridFSIO</code>).</p><p>When the transform should logically return a composite type for which no Java class exists yet, create a new POJO class with well-named fields. Do not use generic tuple classes or <code>KV</code> (unless the fields are legitimately a key and a value).</p><h4 id=transforms-with-multiple-output-collections>Transforms with multiple output collections</h4><p>If the transform needs to return multiple collections, it should be a <code>PTransform&lt;..., PCollectionTuple></code> and expose methods <code>getBlahTag()</code> for each collection.</p><p>E.g. if you want to return a <code>PCollection&lt;Foo></code> and a <code>PCollection&lt;Bar></code>, expose <code>TupleTag&lt;Foo> getFooTag()</code> and <code>TupleTag&lt;Bar> getBarTag()</code>.</p><p>For example:</p><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=kd>public</span> <span class=kd>class</span> <span class=nc>MyTransform</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...,</span> <span class=n>PCollectionTuple</span><span class=o>&gt;</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kd>private</span> <span class=kd>final</span> <span class=n>TupleTag</span><span class=o>&lt;</span><span class=n>Moo</span><span class=o>&gt;</span> <span class=n>mooTag</span> <span class=o>=</span> <span class=k>new</span> <span class=n>TupleTag</span><span class=o>&lt;</span><span class=n>Moo</span><span class=o>&gt;()</span> <span class=o>{};</span>
</span></span><span class=line><span class=cl> <span class=kd>private</span> <span class=kd>final</span> <span class=n>TupleTag</span><span class=o>&lt;</span><span class=n>Blah</span><span class=o>&gt;</span> <span class=n>blahTag</span> <span class=o>=</span> <span class=k>new</span> <span class=n>TupleTag</span><span class=o>&lt;</span><span class=n>Blah</span><span class=o>&gt;()</span> <span class=o>{};</span>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl> <span class=n>PCollectionTuple</span> <span class=nf>expand</span><span class=o>(...</span> <span class=n>input</span><span class=o>)</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl> <span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Moo</span><span class=o>&gt;</span> <span class=n>moo</span> <span class=o>=</span> <span class=o>...;</span>
</span></span><span class=line><span class=cl> <span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Blah</span><span class=o>&gt;</span> <span class=n>blah</span> <span class=o>=</span> <span class=o>...;</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>PCollectionTuple</span><span class=o>.</span><span class=na>of</span><span class=o>(</span><span class=n>mooTag</span><span class=o>,</span> <span class=n>moo</span><span class=o>)</span>
</span></span><span class=line><span class=cl> <span class=o>.</span><span class=na>and</span><span class=o>(</span><span class=n>blahTag</span><span class=o>,</span> <span class=n>blah</span><span class=o>);</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=n>TupleTag</span><span class=o>&lt;</span><span class=n>Moo</span><span class=o>&gt;</span> <span class=nf>getMooTag</span><span class=o>()</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>mooTag</span><span class=o>;</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=n>TupleTag</span><span class=o>&lt;</span><span class=n>Blah</span><span class=o>&gt;</span> <span class=nf>getBlahTag</span><span class=o>()</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>blahTag</span><span class=o>;</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl><span class=o>}</span></span></span></code></pre></div></div></div><h4 id=fluent-builders-for-configuration>Fluent builders for configuration</h4><p>Make the transform class immutable, with methods to produce modified immutable objects. Use <a href=https://github.com/google/auto/tree/master/value>AutoValue</a>. Autovalue can provide a Builder helper class. Use <code>@Nullable</code> to mark parameters of class type that don&rsquo;t have a default value or whose default value is null, except for primitive types (e.g. int).</p><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=nd>@AutoValue</span>
</span></span><span class=line><span class=cl><span class=kd>public</span> <span class=kd>abstract</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>MyTransform</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kt>int</span> <span class=nf>getMoo</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=nd>@Nullable</span> <span class=kd>abstract</span> <span class=n>String</span> <span class=nf>getBlah</span><span class=o>();</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=n>Builder</span> <span class=nf>toBuilder</span><span class=o>();</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=nd>@AutoValue.Builder</span>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Builder</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=n>Builder</span> <span class=nf>setMoo</span><span class=o>(</span><span class=kt>int</span> <span class=n>moo</span><span class=o>);</span>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=n>Builder</span> <span class=nf>setBlah</span><span class=o>(</span><span class=n>String</span> <span class=n>blah</span><span class=o>);</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=n>MyTransform</span> <span class=nf>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl><span class=o>}</span></span></span></code></pre></div></div></div><h5 id=factory-methods>Factory methods</h5><p>Provide a single argumentless static factory method, either in the enclosing class (see &ldquo;Packaging a family of transforms&rdquo;) or in the transform class itself.</p><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=kd>public</span> <span class=kd>class</span> <span class=nc>Thumbs</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>Twiddle</span> <span class=nf>twiddle</span><span class=o>()</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=k>new</span> <span class=n>AutoValue_Thumbs_Twiddle</span><span class=o>.</span><span class=na>Builder</span><span class=o>().</span><span class=na>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>abstract</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Twiddle</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl><span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=c1>// or:
</span></span></span><span class=line><span class=cl><span class=c1></span><span class=kd>public</span> <span class=kd>abstract</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>TwiddleThumbs</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>TwiddleThumbs</span> <span class=nf>create</span><span class=o>()</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=k>new</span> <span class=n>AutoValue_Thumbs_Twiddle</span><span class=o>.</span><span class=na>Builder</span><span class=o>().</span><span class=na>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl><span class=o>}</span></span></span></code></pre></div></div></div><p>Exception: when transform has a single overwhelmingly most important parameter, then call the factory method <code>of</code> and put the parameter into an argument of the factory method: <code>ParDo.of(DoFn).withAllowedLateness()</code>.</p><h5 id=fluent-builder-methods-for-setting-parameters>Fluent builder methods for setting parameters</h5><p>Call them <code>withBlah()</code>. All builder methods must return exactly the same type; if it&rsquo;s a parameterized (generic) type, with the same values of type parameters.</p><p>Treat <code>withBlah()</code> methods as an unordered set of keyword arguments - result must not depend on the order in which you call <code>withFoo()</code> and <code>withBar()</code> (e.g., <code>withBar()</code> must not read the current value of foo).</p><p>Document implications of each <code>withBlah</code> method: when to use this method at all, what values are allowed, what is the default, what are the implications of changing the value.</p><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=cm>/**
</span></span></span><span class=line><span class=cl><span class=cm> * Returns a new {@link TwiddleThumbs} transform with moo set
</span></span></span><span class=line><span class=cl><span class=cm> * to the given value.
</span></span></span><span class=line><span class=cl><span class=cm> *
</span></span></span><span class=line><span class=cl><span class=cm> * &lt;p&gt;Valid values are 0 (inclusive) to 100 (exclusive). The default is 42.
</span></span></span><span class=line><span class=cl><span class=cm> *
</span></span></span><span class=line><span class=cl><span class=cm> * &lt;p&gt;Higher values generally improve throughput, but increase chance
</span></span></span><span class=line><span class=cl><span class=cm> * of spontaneous combustion.
</span></span></span><span class=line><span class=cl><span class=cm> */</span>
</span></span><span class=line><span class=cl><span class=kd>public</span> <span class=n>Twiddle</span> <span class=nf>withMoo</span><span class=o>(</span><span class=kt>int</span> <span class=n>moo</span><span class=o>)</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=n>checkArgument</span><span class=o>(</span><span class=n>moo</span> <span class=o>&gt;=</span> <span class=n>0</span> <span class=o>&amp;&amp;</span> <span class=n>moo</span> <span class=o>&lt;</span> <span class=n>100</span><span class=o>,</span>
</span></span><span class=line><span class=cl> <span class=s>&#34;Thumbs.Twiddle.withMoo() called with an invalid moo of %s. &#34;</span>
</span></span><span class=line><span class=cl> <span class=o>+</span> <span class=s>&#34;Valid values are 0 (inclusive) to 100 (exclusive)&#34;</span><span class=o>,</span>
</span></span><span class=line><span class=cl> <span class=n>moo</span><span class=o>);</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>toBuilder</span><span class=o>().</span><span class=na>setMoo</span><span class=o>(</span><span class=n>moo</span><span class=o>).</span><span class=na>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl><span class=o>}</span></span></span></code></pre></div></div></div><h5 id=default-values-for-parameters>Default values for parameters</h5><p>Specify them in the factory method (factory method returns an object with default values).</p><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=kd>public</span> <span class=kd>class</span> <span class=nc>Thumbs</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>Twiddle</span> <span class=nf>twiddle</span><span class=o>()</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=k>new</span> <span class=n>AutoValue_Thumbs_Twiddle</span><span class=o>.</span><span class=na>Builder</span><span class=o>().</span><span class=na>setMoo</span><span class=o>(</span><span class=n>42</span><span class=o>).</span><span class=na>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl><span class=o>}</span></span></span></code></pre></div></div></div><h5 id=packaging-multiple-parameters-into-a-reusable-object>Packaging multiple parameters into a reusable object</h5><p>If several parameters of the transform are very tightly logically coupled, sometimes it makes sense to encapsulate them into a container object. Use the same guidelines for this container object (make it immutable, use AutoValue with builders, document <code>withBlah()</code> methods, etc.). For an example, see <a href=https://github.com/apache/beam/blob/master/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java>JdbcIO.DataSourceConfiguration</a>.</p><h4 id=transforms-with-type-parameters>Transforms with type parameters</h4><p>All type parameters should be specified explicitly on factory method. Builder methods (<code>withBlah()</code>) should not change the types.</p><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=kd>public</span> <span class=kd>class</span> <span class=nc>Thumbs</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>Twiddle</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;</span> <span class=nf>twiddle</span><span class=o>()</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=k>new</span> <span class=n>AutoValue_Thumbs_Twiddle</span><span class=o>.</span><span class=na>Builder</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;().</span><span class=na>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=nd>@AutoValue</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>abstract</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Twiddle</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;</span>
</span></span><span class=line><span class=cl> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;</span><span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Foo</span><span class=o>&gt;,</span> <span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Bar</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;&gt;&gt;</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=err>…</span>
</span></span><span class=line><span class=cl> <span class=nd>@Nullable</span> <span class=kd>abstract</span> <span class=n>Bar</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;</span> <span class=nf>getBar</span><span class=o>();</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=n>Builder</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;</span> <span class=nf>toBuilder</span><span class=o>();</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=nd>@AutoValue.Builder</span>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Builder</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=err>…</span>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=n>Builder</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;</span> <span class=nf>setBar</span><span class=o>(</span><span class=n>Bar</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;</span> <span class=n>bar</span><span class=o>);</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=n>Twiddle</span><span class=o>&lt;</span><span class=n>T</span><span class=o>&gt;</span> <span class=nf>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=err>…</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl><span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=c1>// User code:
</span></span></span><span class=line><span class=cl><span class=c1></span><span class=n>Thumbs</span><span class=o>.</span><span class=na>Twiddle</span><span class=o>&lt;</span><span class=n>String</span><span class=o>&gt;</span> <span class=n>twiddle</span> <span class=o>=</span> <span class=n>Thumbs</span><span class=o>.&lt;</span><span class=n>String</span><span class=o>&gt;</span><span class=n>twiddle</span><span class=o>();</span>
</span></span><span class=line><span class=cl><span class=c1>// Or:
</span></span></span><span class=line><span class=cl><span class=c1></span><span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Bar</span><span class=o>&lt;</span><span class=n>String</span><span class=o>&gt;&gt;</span> <span class=n>bars</span> <span class=o>=</span> <span class=n>foos</span><span class=o>.</span><span class=na>apply</span><span class=o>(</span><span class=n>Thumbs</span><span class=o>.&lt;</span><span class=n>String</span><span class=o>&gt;</span><span class=n>twiddle</span><span class=o>()</span> <span class=err>…</span> <span class=o>);</span></span></span></code></pre></div></div></div><p>Exception: when the transform has a single most important parameter and this parameter depends on type T, then prefer to put it right into the factory method: e.g. <code>Combine.globally(SerializableFunction&lt;Iterable&lt;V>,V></code>). This improves Java&rsquo;s type inference and allows the user not to specify type parameters explicitly.</p><p>When the transform has more than one type parameter, or if the meaning of the parameter is non-obvious, name the type parameters like <code>SomethingT</code>, e.g.: a PTransform implementing a classifier algorithm and assigning each input element with a label might be typed as <code>Classify&lt;InputT, LabelT></code>.</p><h4 id=injecting-user-specified-behavior>Injecting user-specified behavior</h4><p>If the transform has an aspect of behavior to be customized by a user&rsquo;s code, make a decision as follows:</p><p>Do:</p><ul><li>If possible, just use PTransform composition as an extensibility device - i.e. if the same effect can be achieved by the user applying the transform in their pipeline and composing it with another <code>PTransform</code>, then the transform itself should not be extensible. E.g., a transform that writes JSON objects to a third-party system should take a <code>PCollection&lt;JsonObject></code> (assuming it is possible to provide a <code>Coder</code> for <code>JsonObject</code>), rather than taking a generic <code>PCollection&lt;T></code> and a <code>ProcessFunction&lt;T, JsonObject></code> (anti-example that should be fixed: <code>TextIO</code>).</li><li>If extensibility by user code is necessary inside the transform, pass the user code as a <code>ProcessFunction</code> or define your own serializable function-like type (ideally single-method, for interoperability with Java 8 lambdas). Because Java erases the types of lambdas, you should be sure to have adequate type information even if a raw-type <code>ProcessFunction</code> is provided by the user. See <code>MapElements</code> and <code>FlatMapElements</code> for examples of how to use <code>ProcessFunction</code> and <code>InferableFunction</code> in tandem to provide good support for both lambdas and concrete subclasses with type information.</li></ul><p>Do not:</p><ul><li>Do not use inheritance for extensibility: users should not subclass the <code>PTransform</code> class.</li></ul><h4 id=packaging-a-family-of-transforms>Packaging a family of transforms</h4><p>When developing a family of highly related transforms (e.g. interacting with the same system in different ways, or providing different implementations of the same high-level task), use a top-level class as a namespace, with multiple factory methods returning transforms corresponding to each individual use case.</p><p>The container class must have a private constructor, so it can&rsquo;t be instantiated directly.</p><p>Document common stuff at <code>FooIO</code> level, and each factory method individually.</p><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=cm>/** Transforms for clustering data. */</span>
</span></span><span class=line><span class=cl><span class=kd>public</span> <span class=kd>class</span> <span class=nc>Cluster</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=c1>// Force use of static factory methods.
</span></span></span><span class=line><span class=cl><span class=c1></span> <span class=kd>private</span> <span class=nf>Cluster</span><span class=o>()</span> <span class=o>{}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=cm>/** Returns a new {@link UsingKMeans} transform. */</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>UsingKMeans</span> <span class=nf>usingKMeans</span><span class=o>()</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>Hierarchically</span> <span class=nf>hierarchically</span><span class=o>()</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=cm>/** Clusters data using the K-Means algorithm. */</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>UsingKMeans</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Hierarchically</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl><span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=kd>public</span> <span class=kd>class</span> <span class=nc>FooIO</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=c1>// Force use of static factory methods.
</span></span></span><span class=line><span class=cl><span class=c1></span> <span class=kd>private</span> <span class=nf>FooIO</span><span class=o>()</span> <span class=o>{}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>Read</span> <span class=nf>read</span><span class=o>()</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Read</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Write</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Delete</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Mutate</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl><span class=o>}</span></span></span></code></pre></div></div></div><p>When supporting multiple versions with incompatible APIs, use the version as a namespace-like class too, and put implementations of different API versions in different files.</p><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=c1>// FooIO.java
</span></span></span><span class=line><span class=cl><span class=c1></span><span class=kd>public</span> <span class=kd>class</span> <span class=nc>FooIO</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=c1>// Force use of static factory methods.
</span></span></span><span class=line><span class=cl><span class=c1></span> <span class=kd>private</span> <span class=nf>FooIO</span><span class=o>()</span> <span class=o>{}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>FooV1</span> <span class=nf>v1</span><span class=o>()</span> <span class=o>{</span> <span class=k>return</span> <span class=k>new</span> <span class=n>FooV1</span><span class=o>();</span> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>FooV2</span> <span class=nf>v2</span><span class=o>()</span> <span class=o>{</span> <span class=k>return</span> <span class=k>new</span> <span class=n>FooV2</span><span class=o>();</span> <span class=o>}</span>
</span></span><span class=line><span class=cl><span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=c1>// FooV1.java
</span></span></span><span class=line><span class=cl><span class=c1></span><span class=kd>public</span> <span class=kd>class</span> <span class=nc>FooV1</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=c1>// Force use of static factory methods outside the package.
</span></span></span><span class=line><span class=cl><span class=c1></span> <span class=n>FooV1</span><span class=o>()</span> <span class=o>{}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>Read</span> <span class=nf>read</span><span class=o>()</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Read</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl><span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl><span class=c1>// FooV2.java
</span></span></span><span class=line><span class=cl><span class=c1></span><span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>FooV2</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=c1>// Force use of static factory methods outside the package.
</span></span></span><span class=line><span class=cl><span class=c1></span> <span class=n>FooV2</span><span class=o>()</span> <span class=o>{}</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=n>Read</span> <span class=nf>read</span><span class=o>()</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kd>static</span> <span class=kd>class</span> <span class=nc>Read</span> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;...&gt;</span> <span class=o>{</span> <span class=o>...</span> <span class=o>}</span>
</span></span><span class=line><span class=cl><span class=o>}</span></span></span></code></pre></div></div></div><h3 id=behavior>Behavior</h3><h4 id=immutability>Immutability</h4><ul><li>Transform classes must be immutable: all variables must be private final and themselves immutable (e.g. if it&rsquo;s a list, it must be an <code>ImmutableList</code>).</li><li>Elements of all <code>PCollection</code>s must be immutable.</li></ul><h4 id=serialization>Serialization</h4><p><code>DoFn</code>, <code>PTransform</code>, <code>CombineFn</code> and other instances will be serialized. Keep the amount of serialized data to a minimum: Mark fields that you don&rsquo;t want serialized as <code>transient</code>. Make classes <code>static</code> whenever possible (so that the instance doesn&rsquo;t capture and serialize the enclosing class instance). Note: In some cases this means that you cannot use anonymous classes.</p><h4 id=validation>Validation</h4><ul><li>Validate individual parameters in <code>.withBlah()</code> methods using <code>checkArgument()</code>. Error messages should mention the name of the parameter, the actual value, and the range of valid values.</li><li>Validate parameter combinations and missing required parameters in the <code>PTransform</code>&rsquo;s <code>.expand()</code> method.</li><li>Validate parameters that the <code>PTransform</code> takes from <code>PipelineOptions</code> in the <code>PTransform</code>&rsquo;s <code>.validate(PipelineOptions)</code> method.
These validations will be executed when the pipeline is already fully constructed/expanded and is about to be run with a particular <code>PipelineOptions</code>.
Most <code>PTransform</code>s do not use <code>PipelineOptions</code> and thus don&rsquo;t need a <code>validate()</code> method - instead, they should perform their validation via the two other methods above.</li></ul><div class='language-java snippet'><div class="notebook-skip code-snippet"><a class=copy type=button data-bs-toggle=tooltip data-bs-placement=bottom title="Copy to clipboard"><img src=/images/copy-icon.svg></a><div class=highlight><pre tabindex=0 class=chroma><code class=language-java data-lang=java><span class=line><span class=cl><span class=nd>@AutoValue</span>
</span></span><span class=line><span class=cl><span class=kd>public</span> <span class=kd>abstract</span> <span class=kd>class</span> <span class=nc>TwiddleThumbs</span>
</span></span><span class=line><span class=cl> <span class=kd>extends</span> <span class=n>PTransform</span><span class=o>&lt;</span><span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Foo</span><span class=o>&gt;,</span> <span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Bar</span><span class=o>&gt;&gt;</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=kt>int</span> <span class=nf>getMoo</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=kd>abstract</span> <span class=n>String</span> <span class=nf>getBoo</span><span class=o>();</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl> <span class=c1>// Validating individual parameters
</span></span></span><span class=line><span class=cl><span class=c1></span> <span class=kd>public</span> <span class=n>TwiddleThumbs</span> <span class=nf>withMoo</span><span class=o>(</span><span class=kt>int</span> <span class=n>moo</span><span class=o>)</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=n>checkArgument</span><span class=o>(</span>
</span></span><span class=line><span class=cl> <span class=n>moo</span> <span class=o>&gt;=</span> <span class=n>0</span> <span class=o>&amp;&amp;</span> <span class=n>moo</span> <span class=o>&lt;</span> <span class=n>100</span><span class=o>,</span>
</span></span><span class=line><span class=cl> <span class=s>&#34;Moo must be between 0 (inclusive) and 100 (exclusive), but was: %s&#34;</span><span class=o>,</span>
</span></span><span class=line><span class=cl> <span class=n>moo</span><span class=o>);</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>toBuilder</span><span class=o>().</span><span class=na>setMoo</span><span class=o>(</span><span class=n>moo</span><span class=o>).</span><span class=na>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=n>TwiddleThumbs</span> <span class=nf>withBoo</span><span class=o>(</span><span class=n>String</span> <span class=n>boo</span><span class=o>)</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=n>checkArgument</span><span class=o>(</span><span class=n>boo</span> <span class=o>!=</span> <span class=kc>null</span><span class=o>,</span> <span class=s>&#34;Boo can not be null&#34;</span><span class=o>);</span>
</span></span><span class=line><span class=cl> <span class=n>checkArgument</span><span class=o>(!</span><span class=n>boo</span><span class=o>.</span><span class=na>isEmpty</span><span class=o>(),</span> <span class=s>&#34;Boo can not be empty&#34;</span><span class=o>);</span>
</span></span><span class=line><span class=cl> <span class=k>return</span> <span class=n>toBuilder</span><span class=o>().</span><span class=na>setBoo</span><span class=o>(</span><span class=n>boo</span><span class=o>).</span><span class=na>build</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=nd>@Override</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=kt>void</span> <span class=nf>validate</span><span class=o>(</span><span class=n>PipelineOptions</span> <span class=n>options</span><span class=o>)</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=kt>int</span> <span class=n>woo</span> <span class=o>=</span> <span class=n>options</span><span class=o>.</span><span class=na>as</span><span class=o>(</span><span class=n>TwiddleThumbsOptions</span><span class=o>.</span><span class=na>class</span><span class=o>).</span><span class=na>getWoo</span><span class=o>();</span>
</span></span><span class=line><span class=cl> <span class=n>checkArgument</span><span class=o>(</span>
</span></span><span class=line><span class=cl> <span class=n>woo</span> <span class=o>&gt;</span> <span class=n>getMoo</span><span class=o>(),</span>
</span></span><span class=line><span class=cl> <span class=s>&#34;Woo (%s) must be smaller than moo (%s)&#34;</span><span class=o>,</span>
</span></span><span class=line><span class=cl> <span class=n>woo</span><span class=o>,</span> <span class=n>getMoo</span><span class=o>());</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=nd>@Override</span>
</span></span><span class=line><span class=cl> <span class=kd>public</span> <span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Bar</span><span class=o>&gt;</span> <span class=nf>expand</span><span class=o>(</span><span class=n>PCollection</span><span class=o>&lt;</span><span class=n>Foo</span><span class=o>&gt;</span> <span class=n>input</span><span class=o>)</span> <span class=o>{</span>
</span></span><span class=line><span class=cl> <span class=c1>// Validating that a required parameter is present
</span></span></span><span class=line><span class=cl><span class=c1></span> <span class=n>checkArgument</span><span class=o>(</span><span class=n>getBoo</span><span class=o>()</span> <span class=o>!=</span> <span class=kc>null</span><span class=o>,</span> <span class=s>&#34;Must specify boo&#34;</span><span class=o>);</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=c1>// Validating a combination of parameters
</span></span></span><span class=line><span class=cl><span class=c1></span> <span class=n>checkArgument</span><span class=o>(</span>
</span></span><span class=line><span class=cl> <span class=n>getMoo</span><span class=o>()</span> <span class=o>==</span> <span class=n>0</span> <span class=o>||</span> <span class=n>getBoo</span><span class=o>()</span> <span class=o>==</span> <span class=kc>null</span><span class=o>,</span>
</span></span><span class=line><span class=cl> <span class=s>&#34;Must specify at most one of moo or boo, but was: moo = %s, boo = %s&#34;</span><span class=o>,</span>
</span></span><span class=line><span class=cl> <span class=n>getMoo</span><span class=o>(),</span> <span class=n>getBoo</span><span class=o>());</span>
</span></span><span class=line><span class=cl>
</span></span><span class=line><span class=cl> <span class=o>...</span>
</span></span><span class=line><span class=cl> <span class=o>}</span>
</span></span><span class=line><span class=cl><span class=o>}</span></span></span></code></pre></div></div></div><h4 id=coders>Coders</h4><p><code>Coder</code>s are a way for a Beam runner to materialize intermediate data or transmit it between workers when necessary. <code>Coder</code> should not be used as a general-purpose API for parsing or writing binary formats because the particular binary encoding of a <code>Coder</code> is intended to be its private implementation detail.</p><h5 id=providing-default-coders-for-types>Providing default coders for types</h5><p>Provide default <code>Coder</code>s for all new data types. Use <code>@DefaultCoder</code> annotations or <code>CoderProviderRegistrar</code> classes annotated with <code>@AutoService</code>: see usages of these classes in the SDK for examples. If performance is not important, you can use <code>SerializableCoder</code> or <code>AvroCoder</code>. Otherwise, develop an efficient custom coder (subclass <code>AtomicCoder</code> for concrete types, <code>StructuredCoder</code> for generic types).</p><h5 id=setting-coders-on-output-collections>Setting coders on output collections</h5><p>All <code>PCollection</code>s created by your <code>PTransform</code> (both output and intermediate collections) must have a <code>Coder</code> set on them: a user should never need to call <code>.setCoder()</code> to &ldquo;fix up&rdquo; a coder on a <code>PCollection</code> produced by your <code>PTransform</code> (in fact, Beam intends to eventually deprecate <code>setCoder</code>). In some cases, coder inference will be sufficient to achieve this; in other cases, your transform will need to explicitly call <code>setCoder</code> on its collections.</p><p>If the collection is of a concrete type, that type usually has a corresponding coder. Use a specific most efficient coder (e.g. <code>StringUtf8Coder.of()</code> for strings, <code>ByteArrayCoder.of()</code> for byte arrays, etc.), rather than a general-purpose coder like <code>SerializableCoder</code>.</p><p>If the type of the collection involves generic type variables, the situation is more complex:</p><ul><li>If it coincides with the transform&rsquo;s input type or is a simple wrapper over it, you can reuse the coder of the input <code>PCollection</code>, available via <code>input.getCoder()</code>.</li><li>Attempt to infer the coder via <code>input.getPipeline().getCoderRegistry().getCoder(TypeDescriptor)</code>. Use utilities in <code>TypeDescriptors</code> to obtain the <code>TypeDescriptor</code> for the generic type. For an example of this approach, see the implementation of <code>AvroIO.parseGenericRecords()</code>. However, coder inference for generic types is best-effort and in some cases it may fail due to Java type erasure.</li><li>Always make it possible for the user to explicitly specify a <code>Coder</code> for the relevant type variable(s) as a configuration parameter of your <code>PTransform</code>. (e.g. <code>AvroIO.&lt;T>parseGenericRecords().withCoder(Coder&lt;T>)</code>). Fall back to inference if the coder was not explicitly specified.</li></ul><div class=feedback><p class=update>Last updated on 2024/04/25</p><h3>Have you found everything you were looking for?</h3><p class=description>Was it all useful and clear? Is there anything that you would like to change? Let us know!</p><button class=load-button><a href="https://docs.google.com/forms/d/e/1FAIpQLSfID7abne3GE6k6RdJIyZhPz2Gef7UkpggUEhTIDjjplHuxSA/viewform?usp=header_link" target=_blank>SEND FEEDBACK</a></button></div></div></div><footer class=footer><div class=footer__contained><div class=footer__cols><div class="footer__cols__col footer__cols__col__logos"><div class=footer__cols__col__logo><img src=/images/beam_logo_circle.svg class=footer__logo alt="Beam logo"></div><div class=footer__cols__col__logo><img src=/images/apache_logo_circle.svg class=footer__logo alt="Apache logo"></div></div><div class=footer-wrapper><div class=wrapper-grid><div class=footer__cols__col><div class=footer__cols__col__title>Start</div><div class=footer__cols__col__link><a href=/get-started/beam-overview/>Overview</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-java/>Quickstart (Java)</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-py/>Quickstart (Python)</a></div><div class=footer__cols__col__link><a href=/get-started/quickstart-go/>Quickstart (Go)</a></div><div class=footer__cols__col__link><a href=/get-started/downloads/>Downloads</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Docs</div><div class=footer__cols__col__link><a href=/documentation/programming-guide/>Concepts</a></div><div class=footer__cols__col__link><a href=/documentation/pipelines/design-your-pipeline/>Pipelines</a></div><div class=footer__cols__col__link><a href=/documentation/runners/capability-matrix/>Runners</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Community</div><div class=footer__cols__col__link><a href=/contribute/>Contribute</a></div><div class=footer__cols__col__link><a href=https://projects.apache.org/committee.html?beam target=_blank>Team<img src=/images/external-link-icon.png width=14 height=14 alt="External link."></a></div><div class=footer__cols__col__link><a href=/community/presentation-materials/>Media</a></div><div class=footer__cols__col__link><a href=/community/in-person/>Events/Meetups</a></div><div class=footer__cols__col__link><a href=/community/contact-us/>Contact Us</a></div></div><div class=footer__cols__col><div class=footer__cols__col__title>Resources</div><div class=footer__cols__col__link><a href=/blog/>Blog</a></div><div class=footer__cols__col__link><a href=https://github.com/apache/beam>GitHub</a></div></div></div><div class=footer__bottom>&copy;
<a href=https://www.apache.org>The Apache Software Foundation</a>
| <a href=/privacy_policy>Privacy Policy</a>
| <a href=/feed.xml>RSS Feed</a><br><br>Apache Beam, Apache, Beam, the Beam logo, and the Apache feather logo are either registered trademarks or trademarks of The Apache Software Foundation. All other products or name brands are trademarks of their respective holders, including The Apache Software Foundation.</div></div><div class="footer__cols__col footer__cols__col__logos"><div class=footer__cols__col--group><div class=footer__cols__col__logo><a href=https://github.com/apache/beam><img src=/images/logos/social-icons/github-logo-150.png class=footer__logo alt="Github logo"></a></div><div class=footer__cols__col__logo><a href=https://www.linkedin.com/company/apache-beam/><img src=/images/logos/social-icons/linkedin-logo-150.png class=footer__logo alt="Linkedin logo"></a></div></div><div class=footer__cols__col--group><div class=footer__cols__col__logo><a href=https://twitter.com/apachebeam><img src=/images/logos/social-icons/twitter-logo-150.png class=footer__logo alt="Twitter logo"></a></div><div class=footer__cols__col__logo><a href=https://www.youtube.com/channel/UChNnb_YO_7B0HlW6FhAXZZQ><img src=/images/logos/social-icons/youtube-logo-150.png class=footer__logo alt="Youtube logo"></a></div></div></div></div></div></footer></body></html>