http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/contribute/ptransform-style-guide/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/ptransform-style-guide/index.html 
b/content/contribute/ptransform-style-guide/index.html
new file mode 100644
index 0000000..dbf9b6b
--- /dev/null
+++ b/content/contribute/ptransform-style-guide/index.html
@@ -0,0 +1,759 @@
+<!DOCTYPE html>
+<html lang="en">
+
+  <head>
+  <meta charset="utf-8">
+  <meta http-equiv="X-UA-Compatible" content="IE=edge">
+  <meta name="viewport" content="width=device-width, initial-scale=1">
+
+  <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 rel="stylesheet" href="/styles/site.css">
+  <link rel="stylesheet" href="/css/theme.css">
+  <script 
src="https://ajax.googleapis.com/ajax/libs/jquery/2.2.0/jquery.min.js";></script>
+  <script src="/js/bootstrap.min.js"></script>
+  <script src="/js/language-switch.js"></script>
+  <link rel="canonical" 
href="http://beam.apache.org/contribute/ptransform-style-guide/"; 
data-proofer-ignore>
+  <link rel="alternate" type="application/rss+xml" title="Apache Beam" 
href="http://beam.apache.org/feed.xml";>
+  <script>
+    
(function(i,s,o,g,r,a,m){i['GoogleAnalyticsObject']=r;i[r]=i[r]||function(){
+    (i[r].q=i[r].q||[]).push(arguments)},i[r].l=1*new 
Date();a=s.createElement(o),
+    
m=s.getElementsByTagName(o)[0];a.async=1;a.src=g;m.parentNode.insertBefore(a,m)
+    
})(window,document,'script','//www.google-analytics.com/analytics.js','ga');
+
+    ga('create', 'UA-73650088-1', 'auto');
+    ga('send', 'pageview');
+
+  </script>
+  <link rel="shortcut icon" type="image/x-icon" href="/images/favicon.ico">
+</head>
+
+
+  <body role="document">
+
+    <nav class="navbar navbar-default navbar-fixed-top">
+  <div class="container">
+    <div class="navbar-header">
+      <a href="/" class="navbar-brand" >
+        <img alt="Brand" style="height: 25px" 
src="/images/beam_logo_navbar.png">
+      </a>
+      <button type="button" class="navbar-toggle collapsed" 
data-toggle="collapse" data-target="#navbar" aria-expanded="false" 
aria-controls="navbar">
+        <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 id="navbar" class="navbar-collapse collapse">
+      <ul class="nav navbar-nav">
+        <li class="dropdown">
+                 <a href="#" class="dropdown-toggle" data-toggle="dropdown" 
role="button" aria-haspopup="true" aria-expanded="false">Get Started <span 
class="caret"></span></a>
+                 <ul class="dropdown-menu">
+                         <li><a href="/get-started/beam-overview/">Beam 
Overview</a></li>
+              <li><a href="/get-started/quickstart/">Quickstart</a></li>
+                         <li role="separator" class="divider"></li>
+                         <li class="dropdown-header">Example Walkthroughs</li>
+                         <li><a 
href="/get-started/wordcount-example/">WordCount</a></li>
+                         <li><a 
href="/get-started/mobile-gaming-example/">Mobile Gaming</a></li>
+              <li role="separator" class="divider"></li>
+              <li class="dropdown-header">Resources</li>
+              <li><a href="/get-started/downloads">Downloads</a></li>
+              <li><a href="/get-started/support">Support</a></li>
+                 </ul>
+           </li>
+        <li class="dropdown">
+                 <a href="#" class="dropdown-toggle" data-toggle="dropdown" 
role="button" aria-haspopup="true" aria-expanded="false">Documentation <span 
class="caret"></span></a>
+                 <ul class="dropdown-menu">
+                         <li><a href="/documentation">Using the 
Documentation</a></li>
+                         <li role="separator" class="divider"></li>
+                         <li class="dropdown-header">Beam Concepts</li>
+                         <li><a 
href="/documentation/programming-guide/">Programming Guide</a></li>
+                         <li><a href="/documentation/resources/">Additional 
Resources</a></li>
+                         <li role="separator" class="divider"></li>
+              <li class="dropdown-header">Pipeline Fundamentals</li>
+              <li><a 
href="/documentation/pipelines/design-your-pipeline/">Design Your 
Pipeline</a></li>
+              <li><a 
href="/documentation/pipelines/create-your-pipeline/">Create Your 
Pipeline</a></li>
+              <li><a href="/documentation/pipelines/test-your-pipeline/">Test 
Your Pipeline</a></li>
+              <li role="separator" class="divider"></li>
+                         <li class="dropdown-header">SDKs</li>
+                         <li><a href="/documentation/sdks/java/">Java 
SDK</a></li>
+                         <li><a href="/documentation/sdks/javadoc/0.4.0/" 
target="_blank">Java SDK API Reference <img src="/images/external-link-icon.png"
+                 width="14" height="14"
+                 alt="External link."></a>
+        </li>
+        <li><a href="/documentation/sdks/python/">Python SDK</a></li>
+                         <li role="separator" class="divider"></li>
+                         <li class="dropdown-header">Runners</li>
+                         <li><a 
href="/documentation/runners/capability-matrix/">Capability Matrix</a></li>
+                         <li><a href="/documentation/runners/direct/">Direct 
Runner</a></li>
+                         <li><a href="/documentation/runners/apex/">Apache 
Apex Runner</a></li>
+                         <li><a href="/documentation/runners/flink/">Apache 
Flink Runner</a></li>
+                         <li><a href="/documentation/runners/spark/">Apache 
Spark Runner</a></li>
+                         <li><a href="/documentation/runners/dataflow/">Cloud 
Dataflow Runner</a></li>
+                 </ul>
+           </li>
+        <li class="dropdown">
+                 <a href="#" class="dropdown-toggle" data-toggle="dropdown" 
role="button" aria-haspopup="true" aria-expanded="false">Contribute <span 
class="caret"></span></a>
+                 <ul class="dropdown-menu">
+                         <li><a href="/contribute">Get Started 
Contributing</a></li>
+        <li role="separator" class="divider"></li>
+        <li class="dropdown-header">Guides</li>
+                         <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
+        <li><a href="/contribute/testing/">Testing Guide</a></li>
+        <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
+        <li role="separator" class="divider"></li>
+        <li class="dropdown-header">Technical References</li>
+        <li><a href="/contribute/design-principles/">Design Principles</a></li>
+                         <li><a href="/contribute/work-in-progress/">Ongoing 
Projects</a></li>
+        <li><a href="/contribute/source-repository/">Source 
Repository</a></li>      
+        <li role="separator" class="divider"></li>
+                         <li class="dropdown-header">Promotion</li>
+        <li><a href="/contribute/presentation-materials/">Presentation 
Materials</a></li>
+        <li><a href="/contribute/logos/">Logos and Design</a></li>
+        <li role="separator" class="divider"></li>
+        <li><a href="/contribute/maturity-model/">Maturity Model</a></li>
+        <li><a href="/contribute/team/">Team</a></li>
+                 </ul>
+           </li>
+
+        <li><a href="/blog">Blog</a></li>
+      </ul>
+      <ul class="nav navbar-nav navbar-right">
+        <li class="dropdown">
+          <a href="#" class="dropdown-toggle" 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:24px;">Apache Software Foundation<span 
class="caret"></span></a>
+          <ul class="dropdown-menu dropdown-menu-right">
+            <li><a href="http://www.apache.org/";>ASF Homepage</a></li>
+            <li><a href="http://www.apache.org/licenses/";>License</a></li>
+            <li><a href="http://www.apache.org/security/";>Security</a></li>
+            <li><a 
href="http://www.apache.org/foundation/thanks.html";>Thanks</a></li>
+            <li><a 
href="http://www.apache.org/foundation/sponsorship.html";>Sponsorship</a></li>
+            <li><a 
href="https://www.apache.org/foundation/policies/conduct";>Code of 
Conduct</a></li>
+          </ul>
+        </li>
+      </ul>
+    </div><!--/.nav-collapse -->
+  </div>
+</nav>
+
+
+<link rel="stylesheet" href="">
+
+
+    <div class="container" role="main">
+
+      <div class="row">
+        <h1 id="ptransform-style-guide">PTransform Style Guide</h1>
+
+<p><em>A style guide for writers of new reusable PTransforms.</em></p>
+
+<ul id="markdown-toc">
+  <li><a href="#language-neutral-considerations" 
id="markdown-toc-language-neutral-considerations">Language-neutral 
considerations</a>    <ul>
+      <li><a href="#consistency" 
id="markdown-toc-consistency">Consistency</a></li>
+      <li><a href="#exposing-a-ptransform-vs-something-else" 
id="markdown-toc-exposing-a-ptransform-vs-something-else">Exposing a PTransform 
vs. something else</a></li>
+      <li><a href="#naming" id="markdown-toc-naming">Naming</a></li>
+      <li><a href="#configuration" 
id="markdown-toc-configuration">Configuration</a>        <ul>
+          <li><a href="#what-goes-into-configuration-vs-input-collection" 
id="markdown-toc-what-goes-into-configuration-vs-input-collection">What goes 
into configuration vs. input collection</a></li>
+          <li><a href="#what-parameters-to-expose" 
id="markdown-toc-what-parameters-to-expose">What parameters to expose</a></li>
+        </ul>
+      </li>
+      <li><a href="#error-handling" id="markdown-toc-error-handling">Error 
handling</a>        <ul>
+          <li><a href="#transform-configuration-errors" 
id="markdown-toc-transform-configuration-errors">Transform configuration 
errors</a></li>
+          <li><a href="#runtime-errors-and-data-consistency" 
id="markdown-toc-runtime-errors-and-data-consistency">Runtime errors and data 
consistency</a></li>
+        </ul>
+      </li>
+      <li><a href="#performance" 
id="markdown-toc-performance">Performance</a></li>
+      <li><a href="#documentation" 
id="markdown-toc-documentation">Documentation</a></li>
+      <li><a href="#logging" id="markdown-toc-logging">Logging</a></li>
+      <li><a href="#testing" id="markdown-toc-testing">Testing</a></li>
+      <li><a href="#compatibility" 
id="markdown-toc-compatibility">Compatibility</a></li>
+    </ul>
+  </li>
+  <li><a href="#java-specific-considerations" 
id="markdown-toc-java-specific-considerations">Java specific considerations</a> 
   <ul>
+      <li><a href="#api" id="markdown-toc-api">API</a>        <ul>
+          <li><a href="#choosing-types-of-input-and-output-pcollections" 
id="markdown-toc-choosing-types-of-input-and-output-pcollections">Choosing 
types of input and output PCollection’s</a></li>
+          <li><a href="#transforms-with-multiple-output-collections" 
id="markdown-toc-transforms-with-multiple-output-collections">Transforms with 
multiple output collections</a></li>
+          <li><a href="#fluent-builders-for-configuration" 
id="markdown-toc-fluent-builders-for-configuration">Fluent builders for 
configuration</a>            <ul>
+              <li><a href="#factory-methods" 
id="markdown-toc-factory-methods">Factory methods</a></li>
+              <li><a href="#fluent-builder-methods-for-setting-parameters" 
id="markdown-toc-fluent-builder-methods-for-setting-parameters">Fluent builder 
methods for setting parameters</a></li>
+              <li><a href="#default-values-for-parameters" 
id="markdown-toc-default-values-for-parameters">Default values for 
parameters</a></li>
+              <li><a 
href="#packaging-multiple-parameters-into-a-reusable-object" 
id="markdown-toc-packaging-multiple-parameters-into-a-reusable-object">Packaging
 multiple parameters into a reusable object</a></li>
+            </ul>
+          </li>
+          <li><a href="#transforms-with-type-parameters" 
id="markdown-toc-transforms-with-type-parameters">Transforms with type 
parameters</a></li>
+          <li><a href="#injecting-user-specified-behavior" 
id="markdown-toc-injecting-user-specified-behavior">Injecting user-specified 
behavior</a></li>
+          <li><a href="#packaging-a-family-of-transforms" 
id="markdown-toc-packaging-a-family-of-transforms">Packaging a family of 
transforms</a></li>
+        </ul>
+      </li>
+      <li><a href="#behavior" id="markdown-toc-behavior">Behavior</a>        
<ul>
+          <li><a href="#immutability" 
id="markdown-toc-immutability">Immutability</a></li>
+          <li><a href="#serialization" 
id="markdown-toc-serialization">Serialization</a></li>
+          <li><a href="#validation" 
id="markdown-toc-validation">Validation</a></li>
+          <li><a href="#coders" id="markdown-toc-coders">Coders</a></li>
+        </ul>
+      </li>
+    </ul>
+  </li>
+</ul>
+
+<h2 id="language-neutral-considerations">Language-neutral considerations</h2>
+
+<h3 id="consistency">Consistency</h3>
+<p>Be consistent with prior art:</p>
+
+<ul>
+  <li>Adhere to <a href="/contribute/design-principles/">Beam design 
principles</a>.</li>
+  <li>If there is already a similar transform in some SDK, make the API of 
your transform similar, so that users’ 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>
+</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 class="highlighter-rouge">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 class="highlighter-rouge">ParDo</code> 
can become a more complex transform over time.</li>
+  <li>Expose large, non-trivial, reusable sequential bits of the transform’s 
code, which others might want to reuse in ways you haven’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 
class="highlighter-rouge">CombineFn</code>, or nontrivial <code 
class="highlighter-rouge">DoFn</code>s (those that are more than just a single 
<code class="highlighter-rouge">@ProcessElement</code> method).
+As a rule of thumb: expose these if you anticipate that the full packaged 
<code class="highlighter-rouge">PTransform</code> may be insufficient for a 
user’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 
class="highlighter-rouge">DoFn</code>, concrete <code 
class="highlighter-rouge">Source</code> or <code 
class="highlighter-rouge">Sink</code> classes, etc., in order to avoid 
presenting users with a confusing choice between applying the <code 
class="highlighter-rouge">PTransform</code> or using the <code 
class="highlighter-rouge">DoFn</code>/<code 
class="highlighter-rouge">Source</code>/<code 
class="highlighter-rouge">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 
class="highlighter-rouge">CamelCase</code> in Java and Python, functions in 
<code class="highlighter-rouge">snakeCase</code> in Java but <code 
class="highlighter-rouge">with_underscores</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 
class="highlighter-rouge">MongoDbIO.read()</code>, <code 
class="highlighter-rouge">Flatten.iterables()</code>.</li>
+  <li>In typed languages, name <code 
class="highlighter-rouge">PTransform</code> classes also like verbs (e.g.: 
<code class="highlighter-rouge">MongoDbIO.Read</code>, <code 
class="highlighter-rouge">Flatten.Iterables</code>).</li>
+  <li>Name families of transforms for interacting with a storage system using 
the word “IO”: <code class="highlighter-rouge">MongoDbIO</code>, <code 
class="highlighter-rouge">JdbcIO</code>.</li>
+</ul>
+
+<p>Do not:</p>
+
+<ul>
+  <li>Do not use words <code class="highlighter-rouge">transform</code>, <code 
class="highlighter-rouge">source</code>, <code 
class="highlighter-rouge">sink</code>, <code 
class="highlighter-rouge">reader</code>, <code 
class="highlighter-rouge">writer</code>, <code 
class="highlighter-rouge">bound</code>, <code 
class="highlighter-rouge">unbound</code> in <code 
class="highlighter-rouge">PTransform</code> class names (note: <code 
class="highlighter-rouge">bounded</code> and <code 
class="highlighter-rouge">unbounded</code> are fine when referring to whether a 
<code class="highlighter-rouge">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 
class="highlighter-rouge">PCollection</code>:</strong> anything of which there 
may be a very large number of instances (if there can be &gt;1000 of it, it 
should be in a <code class="highlighter-rouge">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 
class="highlighter-rouge">Bounded</code>/<code 
class="highlighter-rouge">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 
class="highlighter-rouge">ValueProvider</code>s) and does not depend on the 
contents of the transform’s input <code 
class="highlighter-rouge">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’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’s configuration class and let user 
supply a whole instance. Example: <code class="highlighter-rouge">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 “delivery guarantee” 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’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="http://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’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 
class="highlighter-rouge">URL</code> class, rather than the <code 
class="highlighter-rouge">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 
class="highlighter-rouge">PCollection&lt;Document&gt;</code> rather than <code 
class="highlighter-rouge">PCollection&lt;String&gt;</code> (assuming it is 
possible to provide a <code class="highlighter-rouge">Coder</code> for <code 
class="highlighter-rouge">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’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’t be prevented, fail.</p>
+
+<p>Do:</p>
+
+<ul>
+  <li>In a <code class="highlighter-rouge">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’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 a side output.</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’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’t be done.</li>
+</ul>
+
+<p>Do not:</p>
+
+<ul>
+  <li>If you can’t handle a failure, don’t even catch it.
+*Exception: *It may be valuable to catch the error, log a message, and rethrow 
it, if you’re able to provide valuable context that the original error 
doesn’t have.</li>
+  <li>Never, ever, ever do this:
+<code class="highlighter-rouge">catch(...)  { log an error; return null or 
false or otherwise ignore; }</code>
+<strong>Rule of thumb: if a bundle didn’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 
class="highlighter-rouge">ParDo</code>s in ways that improve performance if the 
<code class="highlighter-rouge">ParDo</code>s emit a small to moderate number 
of elements per input element, or have relatively cheap per-element processing 
(e.g. Dataflow’s “fusion”, Apex “compute locality”), but limit 
parallelization if these assumptions are violated. In that case you may need a 
“fusion break” (<code class="highlighter-rouge">Reshuffle.of()</code>) to 
improve the parallelizability of processing the output <code 
class="highlighter-rouge">PCollection</code> of the <code 
class="highlighter-rouge">ParDo</code>.</p>
+
+<ul>
+  <li>If the transform includes a <code class="highlighter-rouge">ParDo</code> 
that outputs a potentially large number of elements per input element, apply a 
fusion break after this <code class="highlighter-rouge">ParDo</code> to make 
sure downstream transforms can process its output in parallel.</li>
+  <li>If the transform includes a <code class="highlighter-rouge">ParDo</code> 
that takes a very long time to process an element, insert a fusion break before 
this <code class="highlighter-rouge">ParDo</code> to make sure all or most 
elements can be processed in parallel regardless of how its input <code 
class="highlighter-rouge">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’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 class="highlighter-rouge">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’s a non-trivial relationship between arrival of items in the 
input <code class="highlighter-rouge">PCollection</code> and emitting output 
into the output <code class="highlighter-rouge">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’s “suppressed 
exceptions”). 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 class="highlighter-rouge">INFO</code> per element or 
per bundle. <code class="highlighter-rouge">DEBUG</code>/<code 
class="highlighter-rouge">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’s hard to check if the output 
is correct, you can’t attach a debugger, and you often can’t log as much as 
you wish to, due to high volume of data. Because of that, testing is 
particularly important.</p>
+
+<ul>
+  <li>Unit-test the overall semantics of the transform using <code 
class="highlighter-rouge">TestPipeline</code> and <code 
class="highlighter-rouge">PAssert</code>. Start with testing against the direct 
runner. Assertions on <code class="highlighter-rouge">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 class="highlighter-rouge">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 
class="highlighter-rouge">TestPipeline</code>, extract this logic into 
unit-testable functions, and unit-test them. Common corner cases are:
+    <ul>
+      <li><code class="highlighter-rouge">DoFn</code>s processing empty 
bundles</li>
+      <li><code class="highlighter-rouge">DoFn</code>s processing extremely 
large bundles (contents doesn’t fit in memory, including “hot keys” 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 class="highlighter-rouge">Closeable</code>/<code 
class="highlighter-rouge">AutoCloseable</code> resources in failure cases</li>
+      <li>Common corner cases when developing sources: complicated arithmetic 
in <code class="highlighter-rouge">BoundedSource.splitIntoBundles</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="http://martinfowler.com/articles/mocksArentStubs.html";>“fake”</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 class="highlighter-rouge">DoFn</code>s, <code 
class="highlighter-rouge">CombineFn</code>s, and <code 
class="highlighter-rouge">BoundedSource</code>s, consider using <code 
class="highlighter-rouge">DoFnTester</code>, <code 
class="highlighter-rouge">CombineFnTester</code>, and <code 
class="highlighter-rouge">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 
class="highlighter-rouge">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="/contribute/testing/">Beam Testing Guide</a>.</li>
+</ul>
+
+<h3 id="compatibility">Compatibility</h3>
+
+<p>Do:</p>
+
+<ul>
+  <li>Generally, follow the rules of <a href="http://semver.org/";>semantic 
versioning</a>.</li>
+  <li>If the API of the transform is not yet stable, mark it <code 
class="highlighter-rouge">@Experimental</code>.</li>
+  <li>Pay attention to the stability and versioning of third-party classes 
exposed by the transform’s API: if they are unstable or improperly versioned 
(do not obey <a href="http://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’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’re merely 
changing implementation or API - do not change API of the transform in a way 
that will make a user’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 
class="highlighter-rouge">JdbcIO</code> and <code 
class="highlighter-rouge">MongoDbIO</code>.</p>
+
+<h3 id="api">API</h3>
+
+<h4 id="choosing-types-of-input-and-output-pcollections">Choosing types of 
input and output PCollection’s</h4>
+
+<p>Whenever possible, use types specific to the nature of the transform. 
People can wrap it with conversion <code class="highlighter-rouge">DoFn</code>s 
from their own types if necessary. E.g. a Datastore connector should use the 
Datastore <code class="highlighter-rouge">Entity</code> type, a MongoDb 
connector should use Mongo <code class="highlighter-rouge">Document</code> 
type, not a String representation of the JSON.</p>
+
+<p>Sometimes that’s not possible (e.g. JDBC does not provide a 
Beam-compatible (encodable with a Coder) “JDBC record” datatype) - then let 
the user provide a function for converting between the transform-specific type 
and a Beam-compatible type (e.g. see <code 
class="highlighter-rouge">JdbcIO</code> and <code 
class="highlighter-rouge">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 class="highlighter-rouge">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 
class="highlighter-rouge">PTransform&lt;..., PCollectionTuple&gt;</code> and 
expose methods <code class="highlighter-rouge">getBlahTag()</code> for each 
collection.</p>
+
+<p>E.g. if you want to return a <code 
class="highlighter-rouge">PCollection&lt;Foo&gt;</code> and a <code 
class="highlighter-rouge">PCollection&lt;Bar&gt;</code>, expose <code 
class="highlighter-rouge">TupleTag&lt;Foo&gt; getFooTag()</code> and <code 
class="highlighter-rouge">TupleTag&lt;Bar&gt; getBarTag()</code>.</p>
+
+<p>For example:</p>
+
+<div class="language-java highlighter-rouge"><pre 
class="highlight"><code><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 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 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 class="o">...</span>
+  <span class="n">PCollectionTuple</span> <span class="nf">apply</span><span 
class="o">(...</span> <span class="n">input</span><span class="o">)</span> 
<span class="o">{</span>
+    <span class="o">...</span>
+    <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 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 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 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 class="o">}</span>
+
+  <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 class="k">return</span> <span class="n">mooTag</span><span 
class="o">;</span>
+  <span class="o">}</span>
+
+  <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 class="k">return</span> <span class="n">blahTag</span><span 
class="o">;</span>
+  <span class="o">}</span>
+  <span class="o">...</span>
+<span class="o">}</span>
+</code></pre>
+</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 
class="highlighter-rouge">@Nullable</code> to mark parameters of class type 
that don’t have a default value or whose default value is null, except for 
primitive types (e.g. int).</p>
+
+<div class="language-java highlighter-rouge"><pre 
class="highlight"><code><span class="nd">@AutoValue</span>
+<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 class="kt">int</span> <span class="nf">getMoo</span><span 
class="o">();</span>
+  <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 class="kd">abstract</span> <span class="n">Builder</span> <span 
class="nf">toBuilder</span><span class="o">();</span>
+
+  <span class="nd">@AutoValue</span><span class="o">.</span><span 
class="na">Builder</span>
+  <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 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 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 class="kd">abstract</span> <span class="n">MyTransform</span> <span 
class="nf">build</span><span class="o">();</span>
+  <span class="o">}</span>
+  <span class="o">...</span>
+<span class="o">}</span>
+</code></pre>
+</div>
+
+<h5 id="factory-methods">Factory methods</h5>
+
+<p>Provide a single argumentless static factory method, either in the 
enclosing class (see “Packaging a family of transforms”) or in the 
transform class itself.</p>
+
+<div class="language-java highlighter-rouge"><pre 
class="highlight"><code><span class="kd">public</span> <span 
class="kd">class</span> <span class="nc">Thumbs</span> <span class="o">{</span>
+  <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 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 class="o">}</span>
+
+  <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 class="o">}</span>
+
+<span class="c1">// or:</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 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 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 class="o">}</span>
+  <span class="o">...</span>
+<span class="o">}</span>
+</code></pre>
+</div>
+
+<p>Exception: when transform has a single overwhelmingly most important 
parameter, then call the factory method <code 
class="highlighter-rouge">of</code> and put the parameter into an argument of 
the factory method: <code 
class="highlighter-rouge">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 class="highlighter-rouge">withBlah()</code>. All builder 
methods must return exactly the same type; if it’s a parameterized (generic) 
type, with the same values of type parameters.</p>
+
+<p>Treat <code class="highlighter-rouge">withBlah()</code> methods as an 
unordered set of keyword arguments - result must not depend on the order in 
which you call <code class="highlighter-rouge">withFoo()</code> and <code 
class="highlighter-rouge">withBar()</code> (e.g., <code 
class="highlighter-rouge">withBar()</code> must not read the current value of 
foo).</p>
+
+<p>Document implications of each <code 
class="highlighter-rouge">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 highlighter-rouge"><pre 
class="highlight"><code><span class="cm">/**
+ * Returns a new {@link TwiddleThumbs} transform with moo set
+ * to the given value.
+ *
+ * &lt;p&gt;Valid values are 0 (inclusive) to 100 (exclusive). The default is 
42.
+ *
+ * &lt;p&gt;Higher values generally improve throughput, but increase chance
+ * of spontaneous combustion.
+ */</span>
+<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 class="n">checkArgument</span><span class="o">(</span><span 
class="n">moo</span> <span class="o">&gt;=</span> <span class="mi">0</span> 
<span class="o">&amp;&amp;</span> <span class="n">moo</span> <span 
class="o">&lt;</span> <span class="mi">100</span><span class="o">,</span>
+      <span class="s">"Thumbs.Twiddle.withMoo() called with an invalid moo of 
%s. "</span>
+      <span class="o">+</span> <span class="s">"Valid values are 0 (inclusive) 
to 100 (exclusive)"</span><span class="o">,</span>
+      <span class="n">moo</span><span class="o">);</span>
+  <span class="k">return</span> <span class="nf">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 class="o">}</span>
+</code></pre>
+</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 highlighter-rouge"><pre 
class="highlight"><code><span class="kd">public</span> <span 
class="kd">class</span> <span class="nc">Thumbs</span> <span class="o">{</span>
+  <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 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="mi">42</span><span 
class="o">).</span><span class="na">build</span><span class="o">();</span>
+  <span class="o">}</span>
+  <span class="o">...</span>
+<span class="o">}</span>
+</code></pre>
+</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 class="highlighter-rouge">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 class="highlighter-rouge">withBlah()</code>) should not 
change the types.</p>
+
+<div class="language-java highlighter-rouge"><pre 
class="highlight"><code><span class="kd">public</span> <span 
class="kd">class</span> <span class="nc">Thumbs</span> <span class="o">{</span>
+  <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 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 class="o">}</span>
+  
+  <span class="nd">@AutoValue</span>
+  <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 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 class="err">…</span>
+    <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 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 class="nd">@AutoValue</span><span class="o">.</span><span 
class="na">Builder</span>
+    <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 class="err">…</span>
+      <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 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 class="o">}</span>
+    <span class="err">…</span>
+  <span class="o">}</span>
+<span class="o">}</span>
+
+<span class="c1">// User code:</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 class="c1">// Or:</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>
+</code></pre>
+</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 
class="highlighter-rouge">Combine.globally(SerializableFunction&lt;Iterable&lt;V&gt;,V&gt;</code>).
 This improves Java’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 
class="highlighter-rouge">SomethingT</code>, e.g.: a PTransform implementing a 
classifier algorithm and assigning each input element with a label might be 
typed as <code class="highlighter-rouge">Classify&lt;InputT, 
LabelT&gt;</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’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 
class="highlighter-rouge">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 
class="highlighter-rouge">PCollection&lt;JsonObject&gt;</code> (assuming it is 
possible to provide a <code class="highlighter-rouge">Coder</code> for <code 
class="highlighter-rouge">JsonObject</code>), rather than taking a generic 
<code class="highlighter-rouge">PCollection&lt;T&gt;</code> and a <code 
class="highlighter-rouge">SerializableFunction&lt;T, JsonObject&gt;</code> 
(anti-example that should be fixed: <code 
class="highlighter-rouge">TextIO</code>).</li>
+  <li>If extensibility by user code is necessary inside the transform, pass 
the user code as a <code class="highlighter-rouge">SerializableFunction</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 class="highlighter-rouge">SerializableFunction</code> is 
provided by the user. See <code class="highlighter-rouge">MapElements</code> 
and <code class="highlighter-rouge">FlatMapElements</code> for examples of how 
to use <code class="highlighter-rouge">SimpleFunction</code> and <code 
class="highlighter-rouge">SerializableFunction</code> in tandem to support Java 
7 and Java 8 well.</li>
+</ul>
+
+<p>Do not:</p>
+
+<ul>
+  <li>Do not use inheritance for extensibility: users should not subclass the 
<code class="highlighter-rouge">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’t be 
instantiated directly.</p>
+
+<p>Document common stuff at <code class="highlighter-rouge">BlahIO</code> 
level, and each factory method individually.</p>
+
+<div class="language-java highlighter-rouge"><pre 
class="highlight"><code><span class="cm">/** Transforms for clustering data. 
*/</span>
+<span class="kd">public</span> <span class="kd">class</span> <span 
class="nc">Cluster</span> <span class="o">{</span>
+  <span class="c1">// Force use of static factory methods.</span>
+  <span class="kd">private</span> <span class="nf">Cluster</span><span 
class="o">()</span> <span class="o">{}</span>
+
+  <span class="cm">/** Returns a new {@link UsingKMeans} transform. */</span>
+  <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 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 class="cm">/** Clusters data using the K-Means algorithm. */</span>
+  <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 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 class="o">}</span>
+
+<span class="kd">public</span> <span class="n">lass</span> <span 
class="n">FooIO</span> <span class="o">{</span>
+  <span class="c1">// Force use of static factory methods.</span>
+  <span class="kd">private</span> <span class="nf">FooIO</span><span 
class="o">()</span> <span class="o">{}</span>
+
+  <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 class="o">...</span>
+
+  <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 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 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 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 class="o">}</span>
+</code></pre>
+</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 highlighter-rouge"><pre 
class="highlight"><code><span class="c1">// FooIO.java</span>
+<span class="kd">public</span> <span class="kd">class</span> <span 
class="nc">FooIO</span> <span class="o">{</span>
+  <span class="c1">// Force use of static factory methods.</span>
+  <span class="kd">private</span> <span class="nf">FooIO</span><span 
class="o">()</span> <span class="o">{}</span>
+
+  <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 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 class="o">}</span>
+
+<span class="c1">// FooV1.java</span>
+<span class="kd">public</span> <span class="kd">class</span> <span 
class="nc">FooV1</span> <span class="o">{</span>
+  <span class="c1">// Force use of static factory methods outside the 
package.</span>
+  <span class="n">FooV1</span><span class="o">()</span> <span 
class="o">{}</span>
+  <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 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 class="o">}</span>
+
+<span class="c1">// FooV2.java</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 class="c1">// Force use of static factory methods outside the 
package.</span>
+  <span class="n">FooV2</span><span class="o">()</span> <span 
class="o">{}</span>
+  <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 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 class="o">}</span>
+</code></pre>
+</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’s a list, it must be an <code 
class="highlighter-rouge">ImmutableList</code>).</li>
+  <li>Elements of all <code class="highlighter-rouge">PCollection</code>s must 
be immutable.</li>
+</ul>
+
+<h4 id="serialization">Serialization</h4>
+
+<p><code class="highlighter-rouge">DoFn</code>, <code 
class="highlighter-rouge">PTransform</code>, <code 
class="highlighter-rouge">CombineFn</code> and other instances will be 
serialized. Keep the amount of serialized data to a minimum: Mark fields that 
you don’t want serialized as <code 
class="highlighter-rouge">transient</code>. Make classes <code 
class="highlighter-rouge">static</code> whenever possible (so that the instance 
doesn’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 
class="highlighter-rouge">.withBlah()</code> methods. Error messages should 
mention the method being called, the actual value and the range of valid 
values.</li>
+  <li>Validate inter-parameter invariants in the <code 
class="highlighter-rouge">PTransform</code>’s <code 
class="highlighter-rouge">.validate()</code> method.</li>
+</ul>
+
+<div class="language-java highlighter-rouge"><pre 
class="highlight"><code><span class="nd">@AutoValue</span>
+<span class="kd">public</span> <span class="kd">abstract</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;</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 class="kd">abstract</span> <span class="kt">int</span> <span 
class="nf">getMoo</span><span class="o">();</span>
+  <span class="kd">abstract</span> <span class="kt">int</span> <span 
class="nf">getBoo</span><span class="o">();</span>
+
+  <span class="o">...</span>
+  <span class="c1">// Validating individual parameters</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 class="n">checkArgument</span><span class="o">(</span><span 
class="n">moo</span> <span class="o">&gt;=</span> <span class="mi">0</span> 
<span class="o">&amp;&amp;</span> <span class="n">moo</span> <span 
class="o">&lt;</span> <span class="mi">100</span><span class="o">,</span>
+      <span class="s">"TwiddleThumbs.withMoo() called with an invalid moo of 
%s. "</span>
+              <span class="o">+</span> <span class="s">"Valid values are 0 
(exclusive) to 100 (exclusive)"</span><span class="o">,</span>
+              <span class="n">moo</span><span class="o">);</span>
+        <span class="k">return</span> <span class="nf">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 class="o">}</span>
+
+  <span class="c1">// Validating cross-parameter invariants</span>
+  <span class="kd">public</span> <span class="kt">void</span> <span 
class="nf">validate</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 class="n">checkArgument</span><span class="o">(</span><span 
class="n">getMoo</span><span class="o">()</span> <span class="o">==</span> 
<span class="mi">0</span> <span class="o">||</span> <span 
class="n">getBoo</span><span class="o">()</span> <span class="o">==</span> 
<span class="mi">0</span><span class="o">,</span>
+      <span class="s">"TwiddleThumbs created with both .withMoo(%s) and 
.withBoo(%s). "</span>
+      <span class="o">+</span> <span class="s">"Only one of these must be 
specified."</span><span class="o">,</span>
+      <span class="n">getMoo</span><span class="o">(),</span> <span 
class="n">getBoo</span><span class="o">());</span>
+  <span class="o">}</span>
+<span class="o">}</span>
+</code></pre>
+</div>
+
+<h4 id="coders">Coders</h4>
+
+<ul>
+  <li>Use <code class="highlighter-rouge">Coder</code>s only for setting the 
coder on a <code class="highlighter-rouge">PCollection</code> or a mutable 
state cell.</li>
+  <li>When available, use a specific most efficient coder for the datatype 
(e.g. <code class="highlighter-rouge">StringUtf8Coder.of()</code> for strings, 
<code class="highlighter-rouge">ByteArrayCoder.of()</code> for byte arrays, 
etc.), rather than using a generic coder like <code 
class="highlighter-rouge">SerializableCoder</code>. Develop efficient coders 
for types that can be elements of <code 
class="highlighter-rouge">PCollection</code>s.</li>
+  <li>Do not use coders as a general serialization or parsing mechanism for 
arbitrary raw byte data. (anti-examples that should be fixed: <code 
class="highlighter-rouge">TextIO</code>, <code 
class="highlighter-rouge">KafkaIO</code>).</li>
+  <li>In general, any transform that outputs a user-controlled type (that is 
not its input type) needs to accept a coder in the transform configuration 
(example: the <code class="highlighter-rouge">Create.of()</code> transform). 
This gives the user the ability to control the coder no matter how the 
transform is structured: e.g., purely letting the user specify the coder on the 
output <code class="highlighter-rouge">PCollection</code> of the transform is 
insufficient in case the transform internally uses intermediate <code 
class="highlighter-rouge">PCollection</code>s of this type.</li>
+</ul>
+
+      </div>
+
+
+    <hr>
+  <div class="row">
+      <div class="col-xs-12">
+          <footer>
+              <p class="text-center">
+                &copy; Copyright
+                <a href="http://www.apache.org";>The Apache Software 
Foundation</a>,
+                2017. All Rights Reserved.
+              </p>
+              <p class="text-center">
+                <a href="/privacy_policy">Privacy Policy</a> |
+                <a href="/feed.xml">RSS Feed</a>
+              </p>
+          </footer>
+      </div>
+  </div>
+  <!-- container div end -->
+</div>
+
+
+  </body>
+
+</html>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/contribute/release-guide/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/release-guide/index.html 
b/content/contribute/release-guide/index.html
index 5ef3dfd..8ed28cd 100644
--- a/content/contribute/release-guide/index.html
+++ b/content/contribute/release-guide/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/contribute/source-repository/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/source-repository/index.html 
b/content/contribute/source-repository/index.html
index 9990b0c..991e2c4 100644
--- a/content/contribute/source-repository/index.html
+++ b/content/contribute/source-repository/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/contribute/team/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/team/index.html 
b/content/contribute/team/index.html
index 796a877..f950b73 100644
--- a/content/contribute/team/index.html
+++ b/content/contribute/team/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/contribute/testing/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/testing/index.html 
b/content/contribute/testing/index.html
index 786d774..7878126 100644
--- a/content/contribute/testing/index.html
+++ b/content/contribute/testing/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/contribute/work-in-progress/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/work-in-progress/index.html 
b/content/contribute/work-in-progress/index.html
index b6e4115..8f1d499 100644
--- a/content/contribute/work-in-progress/index.html
+++ b/content/contribute/work-in-progress/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/index.html b/content/documentation/index.html
index 52a8d8b..a791a12 100644
--- a/content/documentation/index.html
+++ b/content/documentation/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/pipelines/create-your-pipeline/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/pipelines/create-your-pipeline/index.html 
b/content/documentation/pipelines/create-your-pipeline/index.html
index 776a508..bcb3b6b 100644
--- a/content/documentation/pipelines/create-your-pipeline/index.html
+++ b/content/documentation/pipelines/create-your-pipeline/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/pipelines/design-your-pipeline/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/pipelines/design-your-pipeline/index.html 
b/content/documentation/pipelines/design-your-pipeline/index.html
index d7a72c5..57cd016 100644
--- a/content/documentation/pipelines/design-your-pipeline/index.html
+++ b/content/documentation/pipelines/design-your-pipeline/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/pipelines/test-your-pipeline/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/pipelines/test-your-pipeline/index.html 
b/content/documentation/pipelines/test-your-pipeline/index.html
index 53259bd..dc298d5 100644
--- a/content/documentation/pipelines/test-your-pipeline/index.html
+++ b/content/documentation/pipelines/test-your-pipeline/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/programming-guide/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/programming-guide/index.html 
b/content/documentation/programming-guide/index.html
index 83c9664..2810e5d 100644
--- a/content/documentation/programming-guide/index.html
+++ b/content/documentation/programming-guide/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/resources/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/resources/index.html 
b/content/documentation/resources/index.html
index 50a85ad..1206e10 100644
--- a/content/documentation/resources/index.html
+++ b/content/documentation/resources/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/runners/apex/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/apex/index.html 
b/content/documentation/runners/apex/index.html
index 348620c..e6b7491 100644
--- a/content/documentation/runners/apex/index.html
+++ b/content/documentation/runners/apex/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/runners/capability-matrix/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/capability-matrix/index.html 
b/content/documentation/runners/capability-matrix/index.html
index faea11c..af8d80a 100644
--- a/content/documentation/runners/capability-matrix/index.html
+++ b/content/documentation/runners/capability-matrix/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/runners/dataflow/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/dataflow/index.html 
b/content/documentation/runners/dataflow/index.html
index 9ae254c..be4f8d6 100644
--- a/content/documentation/runners/dataflow/index.html
+++ b/content/documentation/runners/dataflow/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/runners/direct/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/direct/index.html 
b/content/documentation/runners/direct/index.html
index ae6cfc7..5356362 100644
--- a/content/documentation/runners/direct/index.html
+++ b/content/documentation/runners/direct/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/runners/flink/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/flink/index.html 
b/content/documentation/runners/flink/index.html
index 399999f..dcf1e9b 100644
--- a/content/documentation/runners/flink/index.html
+++ b/content/documentation/runners/flink/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/runners/spark/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/runners/spark/index.html 
b/content/documentation/runners/spark/index.html
index b383f7a..cec31e7 100644
--- a/content/documentation/runners/spark/index.html
+++ b/content/documentation/runners/spark/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/sdks/java/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/sdks/java/index.html 
b/content/documentation/sdks/java/index.html
index 7f6343d..fee4321 100644
--- a/content/documentation/sdks/java/index.html
+++ b/content/documentation/sdks/java/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/documentation/sdks/python/index.html
----------------------------------------------------------------------
diff --git a/content/documentation/sdks/python/index.html 
b/content/documentation/sdks/python/index.html
index 434da37..2c97da8 100644
--- a/content/documentation/sdks/python/index.html
+++ b/content/documentation/sdks/python/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/get-started/beam-overview/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/beam-overview/index.html 
b/content/get-started/beam-overview/index.html
index 178ff1e..1ae1f09 100644
--- a/content/get-started/beam-overview/index.html
+++ b/content/get-started/beam-overview/index.html
@@ -103,6 +103,7 @@
                          <li><a 
href="/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="/contribute/testing/">Testing Guide</a></li>
         <li><a href="/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="/contribute/ptransform-style-guide/">PTransform Style 
Guide</a></li>
         <li role="separator" class="divider"></li>
         <li class="dropdown-header">Technical References</li>
         <li><a href="/contribute/design-principles/">Design Principles</a></li>

Reply via email to