You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@beam.apache.org by da...@apache.org on 2017/01/31 01:13:12 UTC

[1/5] beam-site git commit: Adds PTransform style guide

Repository: beam-site
Updated Branches:
  refs/heads/asf-site fedd4ab9d -> 2f4d86d31


Adds PTransform style guide


Project: http://git-wip-us.apache.org/repos/asf/beam-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam-site/commit/af370d40
Tree: http://git-wip-us.apache.org/repos/asf/beam-site/tree/af370d40
Diff: http://git-wip-us.apache.org/repos/asf/beam-site/diff/af370d40

Branch: refs/heads/asf-site
Commit: af370d408088debd28794e5191572a388a0d9a50
Parents: fedd4ab
Author: Eugene Kirpichov <ki...@google.com>
Authored: Thu Jan 26 16:46:56 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 17:11:53 2017 -0800

----------------------------------------------------------------------
 src/_includes/header.html                |   1 +
 src/contribute/contribution-guide.md     |   2 +
 src/contribute/index.md                  |   2 +
 src/contribute/ptransform-style-guide.md | 489 ++++++++++++++++++++++++++
 4 files changed, 494 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam-site/blob/af370d40/src/_includes/header.html
----------------------------------------------------------------------
diff --git a/src/_includes/header.html b/src/_includes/header.html
index 4b638df..6ffd1fe 100644
--- a/src/_includes/header.html
+++ b/src/_includes/header.html
@@ -68,6 +68,7 @@
 			  <li><a href="{{ site.baseurl }}/contribute/contribution-guide/">Contribution Guide</a></li>
         <li><a href="{{ site.baseurl }}/contribute/testing/">Testing Guide</a></li>
         <li><a href="{{ site.baseurl }}/contribute/release-guide/">Release Guide</a></li>
+        <li><a href="{{ site.baseurl }}/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="{{ site.baseurl }}/contribute/design-principles/">Design Principles</a></li>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/af370d40/src/contribute/contribution-guide.md
----------------------------------------------------------------------
diff --git a/src/contribute/contribution-guide.md b/src/contribute/contribution-guide.md
index 7341832..83d92fc 100644
--- a/src/contribute/contribution-guide.md
+++ b/src/contribute/contribution-guide.md
@@ -58,6 +58,8 @@ We suggest using [Google Docs](https://docs.google.com/) for sharing designs tha
 ## Code
 To contribute code to Apache Beam, you\u2019ll have to do a few administrative steps once, and then follow a few guidelines for each contribution.
 
+When developing a new `PTransform`, consult the [PTransform Style Guide]({{ site.baseurl }}/contribute/ptransform-style-guide).
+
 ### One-time Setup
 
 #### [Potentially] Submit Contributor License Agreement

http://git-wip-us.apache.org/repos/asf/beam-site/blob/af370d40/src/contribute/index.md
----------------------------------------------------------------------
diff --git a/src/contribute/index.md b/src/contribute/index.md
index 9bfda4a..6101a8a 100644
--- a/src/contribute/index.md
+++ b/src/contribute/index.md
@@ -14,6 +14,8 @@ Learn how to contribute to the Beam project.
 #### [Technical Vision](https://goo.gl/nk5OM0)
 Learn about the technical vision for Beam, including the designs for SDKs and runners, and the development process.
 
+#### [PTransform Style Guide]({{ site.baseurl }}/contribute/ptransform-style-guide)
+Learn about the best practices for developing new PTransforms.
 
 #### Resources
 Resources for contributing to Beam, including mailing lists, the main GitHub repo, and the issues tracker.

http://git-wip-us.apache.org/repos/asf/beam-site/blob/af370d40/src/contribute/ptransform-style-guide.md
----------------------------------------------------------------------
diff --git a/src/contribute/ptransform-style-guide.md b/src/contribute/ptransform-style-guide.md
new file mode 100644
index 0000000..583f264
--- /dev/null
+++ b/src/contribute/ptransform-style-guide.md
@@ -0,0 +1,489 @@
+---
+layout: default
+title: "PTransform Style Guide"
+permalink: /contribute/ptransform-style-guide/
+---
+
+# PTransform Style Guide
+
+_A style guide for writers of new reusable PTransforms._
+
+* TOC
+{:toc}
+
+## Language-neutral considerations
+
+### Consistency
+Be consistent with prior art:
+
+* Adhere to [Beam design principles]({{ site.baseurl }}/contribute/design-principles/).
+* 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.
+*Exception:* 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.
+* When there is no existing similar transform, stay within what is idiomatic within your language of choice (e.g. Java or Python).
+
+### Exposing a PTransform vs. something else
+
+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?
+
+Do:
+
+* Expose every major data-parallel task accomplished by your library as a composite `PTransform`. This allows the structure of the transform to evolve transparently to the code that uses it: e.g. something that started as a `ParDo` can become a more complex transform over time.
+* 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.
+*Example:* 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.
+* In some cases, this may include Beam-specific classes, such as `CombineFn`, or nontrivial `DoFn`s (those that are more than just a single `@ProcessElement` method).
+As a rule of thumb: expose these if you anticipate that the full packaged `PTransform` may be insufficient for a user's needs and the user may want to reuse the lower-level primitive.
+
+Do not:
+
+* Do not expose the exact way the transform is internally structured. E.g.: the public API surface of your library *usually* (with exception of the last bullet above) should not expose `DoFn`, concrete `Source` or `Sink` classes, etc., in order to avoid presenting users with a confusing choice between applying the `PTransform` or using the `DoFn`/`Source`/`Sink`.
+
+### Naming
+
+Do:
+
+* Respect language-specific naming conventions, e.g. name classes in `CamelCase` in Java and Python, functions in `snakeCase` in Java but `with_underscores` in Python, etc.
+* Name factory functions so that either the function name is a verb, or referring to the transform reads like a verb: e.g. `MongoDbIO.read()`, `Flatten.iterables()`.
+* In typed languages, name `PTransform` classes also like verbs (e.g.: `MongoDbIO.Read`, `Flatten.Iterables`).
+* Name families of transforms for interacting with a storage system using the word "IO": `MongoDbIO`, `JdbcIO`.
+
+Do not:
+
+* Do not use words `transform`, `source`, `sink`, `reader`, `writer`, `bound`, `unbound` in `PTransform` class names (note: `bounded` and `unbounded` are fine when referring to whether a `PCollection` is bounded or unbounded): these words are redundant, confusing, obsolete, or name an existing different concept in the SDK.
+
+### Configuration
+
+#### What goes into configuration vs. input collection
+
+* **Into input `PCollection`:** anything of which there may be a very large number of instances (if there can be >1000 of it, it should be in a `PCollection`), 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 `Bounded`/`UnboundedSource` API. If you absolutely have to use such an API, its input can of course go only into transform configuration.
+* **Into transform configuration:** what is constant throughout the transform (including `ValueProvider`s) and does not depend on the contents of the transform's input `PCollection`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.
+
+#### What parameters to expose
+
+Do:
+
+* **Expose** parameters that are necessary to compute the output.
+
+Do not:
+
+* **Do not expose** 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).
+* When developing a connector to a library that has many parameters, **do not mirror each parameter** of the underlying library - if necessary, reuse the underlying library's configuration class and let user supply a whole instance. Example: `JdbcIO`.
+*Exception 1:* 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).
+*Exception 2:* 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 [semantic versioning](http://semver.org/) - in this case, it is better to wrap it and expose a cleaner and more stable API to users of the transform.
+
+### Error handling
+
+#### Transform configuration errors
+
+Detect errors early. Errors can be detected at the following stages:
+
+* (in a compiled language) compilation of the source code of a user's pipeline
+* constructing or setting up the transform
+* applying the transform in a pipeline
+* running the pipeline
+
+For example:
+
+* In a typed language, take advantage of compile-time error checking by making the API of the transform strongly-typed:
+    * **Strongly-typed configuration:** e.g. in Java, a parameter that is a URL should use the `URL` class, rather than the `String` class.
+    * **Strongly-typed input and output:** e.g. a transform that writes to Mongo DB should take a `PCollection<Document>` rather than `PCollection<String>` (assuming it is possible to provide a `Coder` for `Document`).
+* Detect invalid values of individual parameters in setter methods.
+* Detect invalid combinations of parameters in the transform's validate method.
+
+#### Runtime errors and data consistency
+
+Favor data consistency above everything else. Do not mask data loss or corruption. If data loss can't be prevented, fail.
+
+Do:
+
+* In a `DoFn`, 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 *some* retrying).
+* 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.
+* If the transform can have unprocessable (permanently failing) records and you want the pipeline to proceed despite that:
+    * 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.
+    * If bad records may need manual inspection by the user, emit them into a side output.
+    * 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).
+* 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.
+
+Do not:
+
+* 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.
+* Never, ever, ever do this:
+`catch(...)  { log an error; return null or false or otherwise ignore; }`
+**Rule of thumb: if a bundle didn't fail, its output must be correct and complete.**
+For a user, a transform that logged an error but succeeded is silent data loss.
+
+### Performance
+
+Many runners optimize chains of `ParDo`s in ways that improve performance if the `ParDo`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" (`Reshuffle.of()`) to improve the parallelizability of processing the output `PCollection` of the `ParDo`.
+
+* If the transform includes a `ParDo` that outputs a potentially large number of elements per input element, apply a fusion break after this `ParDo` to make sure downstream transforms can process its output in parallel.
+* If the transform includes a `ParDo` that takes a very long time to process an element, insert a fusion break before this `ParDo` to make sure all or most elements can be processed in parallel regardless of how its input `PCollection` was produced.
+
+### Documentation
+
+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.:
+
+* Are the input and output collections of this transform bounded or unbounded, or can it work with either?
+* 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?)
+* 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.
+* If the transform is querying an external system during processing (e.g. joining a `PCollection` 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)?
+* If there's a non-trivial relationship between arrival of items in the input `PCollection` and emitting output into the output `PCollection`, what is this relationship? (e.g. if the transform internally does windowing, triggering, grouping, or uses the state or timers API)
+
+### Logging
+
+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.
+
+Do:
+
+* 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.
+* 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.
+* 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.
+
+Do not:
+
+* Do not log at `INFO` per element or per bundle. `DEBUG`/`TRACE` may be okay because these levels are disabled by default.
+* Avoid logging data payloads that may contain sensitive information, or sanitize them before logging (e.g. user data, credentials, etc).
+
+### Testing
+
+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.
+
+* Unit-test the overall semantics of the transform using `TestPipeline` and `PAssert`. Start with testing against the direct runner. Assertions on `PCollection` 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 `PCollection`, or that each element is in the range [1, 10] - but assert that each number 1 through 10 appears exactly once.
+* Identify non-trivial sequential logic in the transform that is prone to corner cases which are difficult to reliably simulate using a `TestPipeline`, extract this logic into unit-testable functions, and unit-test them. Common corner cases are:
+    * `DoFn`s processing empty bundles
+    * `DoFn`s processing extremely large bundles (contents doesn't fit in memory, including "hot keys" with a very large number of values)
+    * Third-party APIs failing
+    * Third-party APIs providing wildly inaccurate information
+    * Leaks of `Closeable`/`AutoCloseable` resources in failure cases
+    * Common corner cases when developing sources: complicated arithmetic in `BoundedSource.splitIntoBundles` (e.g. splitting key or offset ranges), iteration over empty data sources or composite data sources that have some empty components.
+* Mock out the interactions with third-party systems, or better, use ["fake"](http://martinfowler.com/articles/mocksArentStubs.html) implementations when available. Make sure that the mocked-out interactions are representative of all interesting cases of the actual behavior of these systems.
+* To unit test `DoFn`s, `CombineFn`s, and `BoundedSource`s, consider using `DoFnTester`, `CombineFnTester`, and `SourceTestUtils` respectively which can exercise the code in non-trivial ways to flesh out potential bugs. 
+* For transforms that work over unbounded collections, test their behavior in the presence of late or out-of-order data using `TestStream`.
+* 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.
+* For detailed instructions on test code organization, see the [Beam Testing Guide]({{ site.baseurl }}/contribute/testing/).
+
+### Compatibility
+
+Do:
+
+* Generally, follow the rules of [semantic versioning](http://semver.org/).
+* If the API of the transform is not yet stable, mark it `@Experimental`.
+* 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 [semantic versioning](http://semver.org/)), it is better to wrap them in your own classes.
+
+Do not:
+
+* 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.
+* 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.
+
+## Java specific considerations
+
+Good examples for most of the practices below are `JdbcIO` and `MongoDbIO`.
+
+### API
+
+#### Choosing types of input and output PCollection's
+
+Whenever possible, use types specific to the nature of the transform. People can wrap it with conversion `DoFn`s from their own types if necessary. E.g. a Datastore connector should use the Datastore `Entity` type, a MongoDb connector should use Mongo `Document` type, not a String representation of the JSON.
+
+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 `JdbcIO` and `MongoDbGridFSIO`).
+
+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 `KV` (unless the fields are legitimately a key and a value).
+
+#### Transforms with multiple output collections
+
+If the transform needs to return multiple collections, it should be a `PTransform<..., PCollectionTuple>` and expose methods `getBlahTag()` for each collection.
+
+E.g. if you want to return a `PCollection<Foo>` and a `PCollection<Bar>`, expose `TupleTag<Foo> getFooTag()` and `TupleTag<Bar> getBarTag()`.
+
+For example:
+
+```java
+public class MyTransform extends PTransform<..., PCollectionTuple> {
+  private final TupleTag<Moo> mooTag = new TupleTag<Moo>() {};
+  private final TupleTag<Blah> blahTag = new TupleTag<Blah>() {};
+  ...
+  PCollectionTuple apply(... input) {
+    ...
+    PCollection<Moo> moo = ...;
+    PCollection<Blah> blah = ...;
+    return PCollectionTuple.of(mooTag, moo)
+                           .and(blahTag, blah);
+  }
+
+  public TupleTag<Moo> getMooTag() {
+    return mooTag;
+  }
+
+  public TupleTag<Blah> getBlahTag() {
+    return blahTag;
+  }
+  ...
+}
+```
+
+#### Fluent builders for configuration
+
+Make the transform class immutable, with methods to produce modified immutable objects. Use [AutoValue](https://github.com/google/auto/tree/master/value). Autovalue can provide a Builder helper class. Use `@Nullable` 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).
+
+```java
+@AutoValue
+public abstract static class MyTransform extends PTransform<...> {
+  int getMoo();
+  @Nullable abstract String getBlah();
+
+  abstract Builder toBuilder();
+
+  @AutoValue.Builder
+  abstract static class Builder {
+    abstract Builder setMoo(int moo);
+    abstract Builder setBlah(String blah);
+
+    abstract MyTransform build();
+  }
+  ...
+}
+```
+
+##### Factory methods
+
+Provide a single argumentless static factory method, either in the enclosing class (see "Packaging a family of transforms") or in the transform class itself.
+
+```java
+public class Thumbs {
+  public static Twiddle twiddle() {
+    return new AutoValue_Thumbs_Twiddle.Builder().build();
+  }
+
+  public abstract static class Twiddle extends PTransform<...> { ... }
+}
+
+// or:
+public abstract static class TwiddleThumbs extends PTransform<...> {
+  public static TwiddleThumbs create() {
+    return new AutoValue_Thumbs_Twiddle.Builder().build();
+  }
+  ...
+}
+```
+
+
+Exception: when transform has a single overwhelmingly most important parameter, then call the factory method `of` and put the parameter into an argument of the factory method: `ParDo.of(DoFn).withAllowedLateness()`.
+
+##### Fluent builder methods for setting parameters
+
+Call them `withBlah()`. All builder methods must return exactly the same type; if it's a parameterized (generic) type, with the same values of type parameters.
+
+Treat `withBlah()` methods as an unordered set of keyword arguments - result must not depend on the order in which you call `withFoo()` and `withBar()` (e.g., `withBar()` must not read the current value of foo).
+
+Document implications of each `withBlah` method: when to use this method at all, what values are allowed, what is the default, what are the implications of changing the value.
+
+```java
+/**
+ * Returns a new {@link TwiddleThumbs} transform with moo set
+ * to the given value.
+ *
+ * <p>Valid values are 0 (inclusive) to 100 (exclusive). The default is 42.
+ *
+ * <p>Higher values generally improve throughput, but increase chance
+ * of spontaneous combustion.
+ */
+public Twiddle withMoo(int moo) {
+  checkArgument(moo >= 0 && moo < 100,
+      "Thumbs.Twiddle.withMoo() called with an invalid moo of %s. "
+      + "Valid values are 0 (inclusive) to 100 (exclusive)",
+      moo);
+  return toBuilder().setMoo(moo).build();
+}
+```
+
+##### Default values for parameters
+
+Specify them in the factory method (factory method returns an object with default values).
+
+```java
+public class Thumbs {
+  public static Twiddle twiddle() {
+    return new AutoValue_Thumbs_Twiddle.Builder().setMoo(42).build();
+  }
+  ...
+}
+```
+
+##### Packaging multiple parameters into a reusable object
+
+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 `withBlah()` methods, etc.). For an example, see [JdbcIO.DataSourceConfiguration](https://github.com/apache/beam/blob/master/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcIO.java).
+
+#### Transforms with type parameters
+
+All type parameters should be specified explicitly on factory method. Builder methods (`withBlah()`) should not change the types.
+
+```java
+public class Thumbs {
+  public static Twiddle<T> twiddle() {
+    return new AutoValue_Thumbs_Twiddle.Builder<T>().build();
+  }
+  
+  @AutoValue
+  public abstract static class Twiddle<T>
+       extends PTransform<PCollection<Foo>, PCollection<Bar<T>>> {
+    \u2026
+    @Nullable abstract Bar<T> getBar();
+
+    abstract Builder<T> toBuilder();
+
+    @AutoValue.Builder
+    abstract static class Builder<T> {
+      \u2026
+      abstract Builder<T> setBar(Bar<T> bar);
+
+      abstract Twiddle<T> build();
+    }
+    \u2026
+  }
+}
+
+// User code:
+Thumbs.Twiddle<String> twiddle = Thumbs.<String>twiddle();
+// Or:
+PCollection<Bar<String>> bars = foos.apply(Thumbs.<String>twiddle() \u2026 );
+```
+
+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. `Combine.globally(SerializableFunction<Iterable<V>,V>`). This improves Java's type inference and allows the user not to specify type parameters explicitly.
+
+When the transform has more than one type parameter, or if the meaning of the parameter is non-obvious, name the type parameters like `SomethingT`, e.g.: a PTransform implementing a classifier algorithm and assigning each input element with a label might be typed as `Classify<InputT, LabelT>`.
+
+#### Injecting user-specified behavior
+
+If the transform has an aspect of behavior to be customized by a user's code, make a decision as follows:
+
+Do:
+
+* 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 `PTransform`, then the transform itself should not be extensible. E.g., a transform that writes JSON objects to a third-party system should take a `PCollection<JsonObject>` (assuming it is possible to provide a `Coder` for `JsonObject`), rather than taking a generic `PCollection<T>` and a `SerializableFunction<T, JsonObject>` (anti-example that should be fixed: `TextIO`).
+* If extensibility by user code is necessary inside the transform, pass the user code as a `SerializableFunction` 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 `SerializableFunction` is provided by the user. See `MapElements` and `FlatMapElements` for examples of how to use `SimpleFunction` and `SerializableFunction` in tandem to support Java 7 and Java 8 well.
+
+Do not:
+
+* Do not use inheritance for extensibility: users should not subclass the `PTransform` class.
+
+#### Packaging a family of transforms
+
+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.
+
+The container class must have a private constructor, so it can't be instantiated directly.
+
+Document common stuff at `BlahIO` level, and each factory method individually.
+
+```java
+/** Transforms for clustering data. */
+public class Cluster {
+  // Force use of static factory methods.
+  private Cluster() {}
+
+  /** Returns a new {@link UsingKMeans} transform. */
+  public static UsingKMeans usingKMeans() { ... }
+  public static Hierarchically hierarchically() { ... }
+
+  /** Clusters data using the K-Means algorithm. */
+  public static class UsingKMeans extends PTransform<...> { ... }
+  public static class Hierarchically extends PTransform<...> { ... }
+}
+
+public lass FooIO {
+  // Force use of static factory methods.
+  private FooIO() {}
+
+  public static Read read() { ... }
+  ...
+
+  public static class Read extends PTransform<...> { ... }
+  public static class Write extends PTransform<...> { ... }
+  public static class Delete extends PTransform<...> { ... }
+  public static class Mutate extends PTransform<...> { ... }
+}
+```
+
+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.
+
+```java
+// FooIO.java
+public class FooIO {
+  // Force use of static factory methods.
+  private FooIO() {}
+
+  public static FooV1 v1() { return new FooV1(); }
+  public static FooV2 v2() { return new FooV2(); }
+}
+
+// FooV1.java
+public class FooV1 {
+  // Force use of static factory methods outside the package.
+  FooV1() {}
+  public static Read read() { ... }
+  public static class Read extends PTransform<...> { ... }
+}
+
+// FooV2.java
+public static class FooV2 {
+  // Force use of static factory methods outside the package.
+  FooV2() {}
+  public static Read read() { ... }
+
+  public static class Read extends PTransform<...> { ... }
+}
+```
+
+### Behavior
+
+#### Immutability
+
+* 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 `ImmutableList`).
+* Elements of all `PCollection`s must be immutable.
+
+#### Serialization
+
+`DoFn`, `PTransform`, `CombineFn` and other instances will be serialized. Keep the amount of serialized data to a minimum: Mark fields that you don't want serialized as `transient`. Make classes `static` 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.
+
+#### Validation
+
+* Validate individual parameters in `.withBlah()` methods. Error messages should mention the method being called, the actual value and the range of valid values.
+* Validate inter-parameter invariants in the `PTransform`'s `.validate()` method.
+
+```java
+@AutoValue
+public abstract class TwiddleThumbs
+    extends PTransform<PCollection<Foo>, PCollection<Bar>> {
+  abstract int getMoo();
+  abstract int getBoo();
+
+  ...
+  // Validating individual parameters
+  public TwiddleThumbs withMoo(int moo) {
+    checkArgument(moo >= 0 && moo < 100,
+      "TwiddleThumbs.withMoo() called with an invalid moo of %s. "
+              + "Valid values are 0 (exclusive) to 100 (exclusive)",
+              moo);
+        return toBuilder().setMoo(moo).build();
+  }
+
+  // Validating cross-parameter invariants
+  public void validate(PCollection<Foo> input) {
+    checkArgument(getMoo() == 0 || getBoo() == 0,
+      "TwiddleThumbs created with both .withMoo(%s) and .withBoo(%s). "
+      + "Only one of these must be specified.",
+      getMoo(), getBoo());
+  }
+}
+```
+
+#### Coders
+
+* Use `Coder`s only for setting the coder on a `PCollection` or a mutable state cell.
+* When available, use a specific most efficient coder for the datatype (e.g. `StringUtf8Coder.of()` for strings, `ByteArrayCoder.of()` for byte arrays, etc.), rather than using a generic coder like `SerializableCoder`. Develop efficient coders for types that can be elements of `PCollection`s.
+* Do not use coders as a general serialization or parsing mechanism for arbitrary raw byte data. (anti-examples that should be fixed: `TextIO`, `KafkaIO`).
+* 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 `Create.of()` 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 `PCollection` of the transform is insufficient in case the transform internally uses intermediate `PCollection`s of this type.


[4/5] beam-site git commit: Regenerate website

Posted by da...@apache.org.
Regenerate website


Project: http://git-wip-us.apache.org/repos/asf/beam-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam-site/commit/077a1747
Tree: http://git-wip-us.apache.org/repos/asf/beam-site/tree/077a1747
Diff: http://git-wip-us.apache.org/repos/asf/beam-site/diff/077a1747

Branch: refs/heads/asf-site
Commit: 077a1747e6ef2f4eed23e558e6aa3022a85dc305
Parents: af370d4
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 17:12:46 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 17:12:46 2017 -0800

----------------------------------------------------------------------
 .../2016/03/17/capability-matrix.html           |   1 +
 .../2016/04/03/presentation-materials.html      |   1 +
 .../sdk/2016/02/25/python-sdk-now-public.html   |   1 +
 .../beam/release/2016/06/15/first-release.html  |   1 +
 .../10/11/strata-hadoop-world-and-beam.html     |   1 +
 .../website/2016/02/22/beam-has-a-logo.html     |   1 +
 .../blog/2016/05/18/splitAtFraction-method.html |   1 +
 .../05/27/where-is-my-pcollection-dot-map.html  |   1 +
 .../06/13/flink-batch-runner-milestone.html     |   1 +
 content/blog/2016/08/03/six-months.html         |   1 +
 content/blog/2016/10/20/test-stream.html        |   1 +
 content/blog/2017/01/09/added-apex-runner.html  |   1 +
 content/blog/2017/01/10/beam-graduates.html     |   1 +
 content/blog/index.html                         |   1 +
 content/coming-soon.html                        |   1 +
 .../contribute/contribution-guide/index.html    |   3 +
 content/contribute/design-principles/index.html |   1 +
 content/contribute/index.html                   |   4 +
 content/contribute/logos/index.html             |   1 +
 content/contribute/maturity-model/index.html    |   1 +
 .../presentation-materials/index.html           |   1 +
 .../ptransform-style-guide/index.html           | 759 +++++++++++++++++++
 content/contribute/release-guide/index.html     |   1 +
 content/contribute/source-repository/index.html |   1 +
 content/contribute/team/index.html              |   1 +
 content/contribute/testing/index.html           |   1 +
 content/contribute/work-in-progress/index.html  |   1 +
 content/documentation/index.html                |   1 +
 .../pipelines/create-your-pipeline/index.html   |   1 +
 .../pipelines/design-your-pipeline/index.html   |   1 +
 .../pipelines/test-your-pipeline/index.html     |   1 +
 .../documentation/programming-guide/index.html  |   1 +
 content/documentation/resources/index.html      |   1 +
 content/documentation/runners/apex/index.html   |   1 +
 .../runners/capability-matrix/index.html        |   1 +
 .../documentation/runners/dataflow/index.html   |   1 +
 content/documentation/runners/direct/index.html |   1 +
 content/documentation/runners/flink/index.html  |   1 +
 content/documentation/runners/spark/index.html  |   1 +
 content/documentation/sdks/java/index.html      |   1 +
 content/documentation/sdks/python/index.html    |   1 +
 content/get-started/beam-overview/index.html    |   1 +
 content/get-started/downloads/index.html        |   1 +
 content/get-started/index.html                  |   1 +
 .../mobile-gaming-example/index.html            |   1 +
 content/get-started/quickstart/index.html       |   1 +
 content/get-started/support/index.html          |   1 +
 .../get-started/wordcount-example/index.html    |   1 +
 content/index.html                              |   1 +
 content/privacy_policy/index.html               |   1 +
 50 files changed, 813 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/beam/capability/2016/03/17/capability-matrix.html
----------------------------------------------------------------------
diff --git a/content/beam/capability/2016/03/17/capability-matrix.html b/content/beam/capability/2016/03/17/capability-matrix.html
index 285b4b4..17e9269 100644
--- a/content/beam/capability/2016/03/17/capability-matrix.html
+++ b/content/beam/capability/2016/03/17/capability-matrix.html
@@ -102,6 +102,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/beam/capability/2016/04/03/presentation-materials.html
----------------------------------------------------------------------
diff --git a/content/beam/capability/2016/04/03/presentation-materials.html b/content/beam/capability/2016/04/03/presentation-materials.html
index 875fb7c..61e0377 100644
--- a/content/beam/capability/2016/04/03/presentation-materials.html
+++ b/content/beam/capability/2016/04/03/presentation-materials.html
@@ -102,6 +102,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/beam/python/sdk/2016/02/25/python-sdk-now-public.html
----------------------------------------------------------------------
diff --git a/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html b/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html
index 25b34a7..4bc8bfe 100644
--- a/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html
+++ b/content/beam/python/sdk/2016/02/25/python-sdk-now-public.html
@@ -102,6 +102,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/beam/release/2016/06/15/first-release.html
----------------------------------------------------------------------
diff --git a/content/beam/release/2016/06/15/first-release.html b/content/beam/release/2016/06/15/first-release.html
index 86e3239..8dd43c4 100644
--- a/content/beam/release/2016/06/15/first-release.html
+++ b/content/beam/release/2016/06/15/first-release.html
@@ -102,6 +102,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/beam/update/2016/10/11/strata-hadoop-world-and-beam.html
----------------------------------------------------------------------
diff --git a/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html b/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html
index 23dd08a..455271f 100644
--- a/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html
+++ b/content/beam/update/2016/10/11/strata-hadoop-world-and-beam.html
@@ -102,6 +102,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/beam/update/website/2016/02/22/beam-has-a-logo.html
----------------------------------------------------------------------
diff --git a/content/beam/update/website/2016/02/22/beam-has-a-logo.html b/content/beam/update/website/2016/02/22/beam-has-a-logo.html
index 06edc02..09ded66 100644
--- a/content/beam/update/website/2016/02/22/beam-has-a-logo.html
+++ b/content/beam/update/website/2016/02/22/beam-has-a-logo.html
@@ -102,6 +102,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/blog/2016/05/18/splitAtFraction-method.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/05/18/splitAtFraction-method.html b/content/blog/2016/05/18/splitAtFraction-method.html
index 6f35e24..3096435 100644
--- a/content/blog/2016/05/18/splitAtFraction-method.html
+++ b/content/blog/2016/05/18/splitAtFraction-method.html
@@ -102,6 +102,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/blog/2016/05/27/where-is-my-pcollection-dot-map.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html b/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html
index de91d2c..4b0652f 100644
--- a/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html
+++ b/content/blog/2016/05/27/where-is-my-pcollection-dot-map.html
@@ -102,6 +102,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/blog/2016/06/13/flink-batch-runner-milestone.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/06/13/flink-batch-runner-milestone.html b/content/blog/2016/06/13/flink-batch-runner-milestone.html
index 72bf3cc..d3bc2f4 100644
--- a/content/blog/2016/06/13/flink-batch-runner-milestone.html
+++ b/content/blog/2016/06/13/flink-batch-runner-milestone.html
@@ -102,6 +102,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/blog/2016/08/03/six-months.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/08/03/six-months.html b/content/blog/2016/08/03/six-months.html
index b57165f..d183491 100644
--- a/content/blog/2016/08/03/six-months.html
+++ b/content/blog/2016/08/03/six-months.html
@@ -102,6 +102,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/blog/2016/10/20/test-stream.html
----------------------------------------------------------------------
diff --git a/content/blog/2016/10/20/test-stream.html b/content/blog/2016/10/20/test-stream.html
index 74cd7ce..211834f 100644
--- a/content/blog/2016/10/20/test-stream.html
+++ b/content/blog/2016/10/20/test-stream.html
@@ -102,6 +102,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/blog/2017/01/09/added-apex-runner.html
----------------------------------------------------------------------
diff --git a/content/blog/2017/01/09/added-apex-runner.html b/content/blog/2017/01/09/added-apex-runner.html
index 9eaa55b..6cc8d6c 100644
--- a/content/blog/2017/01/09/added-apex-runner.html
+++ b/content/blog/2017/01/09/added-apex-runner.html
@@ -102,6 +102,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/blog/2017/01/10/beam-graduates.html
----------------------------------------------------------------------
diff --git a/content/blog/2017/01/10/beam-graduates.html b/content/blog/2017/01/10/beam-graduates.html
index a7fcbef..dab7f53 100644
--- a/content/blog/2017/01/10/beam-graduates.html
+++ b/content/blog/2017/01/10/beam-graduates.html
@@ -102,6 +102,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/blog/index.html
----------------------------------------------------------------------
diff --git a/content/blog/index.html b/content/blog/index.html
index 624302a..0ede785 100644
--- a/content/blog/index.html
+++ b/content/blog/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/coming-soon.html
----------------------------------------------------------------------
diff --git a/content/coming-soon.html b/content/coming-soon.html
index 8bf90b9..4369047 100644
--- a/content/coming-soon.html
+++ b/content/coming-soon.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/contribution-guide/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/contribution-guide/index.html b/content/contribute/contribution-guide/index.html
index 0f9ec0a..14a21a1 100644
--- a/content/contribute/contribution-guide/index.html
+++ b/content/contribute/contribution-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>
@@ -259,6 +260,8 @@
 <h2 id="code">Code</h2>
 <p>To contribute code to Apache Beam, you\u2019ll have to do a few administrative steps once, and then follow a few guidelines for each contribution.</p>
 
+<p>When developing a new <code class="highlighter-rouge">PTransform</code>, consult the <a href="/contribute/ptransform-style-guide">PTransform Style Guide</a>.</p>
+
 <h3 id="one-time-setup">One-time Setup</h3>
 
 <h4 id="potentially-submit-contributor-license-agreement">[Potentially] Submit Contributor License Agreement</h4>

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/contribute/design-principles/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/design-principles/index.html b/content/contribute/design-principles/index.html
index f4f092e..046cbd7 100644
--- a/content/contribute/design-principles/index.html
+++ b/content/contribute/design-principles/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/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/index.html b/content/contribute/index.html
index b8e9cb6..e5303e8 100644
--- a/content/contribute/index.html
+++ b/content/contribute/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>
@@ -154,6 +155,9 @@
 <h4 id="technical-visionhttpsgooglnk5om0"><a href="https://goo.gl/nk5OM0">Technical Vision</a></h4>
 <p>Learn about the technical vision for Beam, including the designs for SDKs and runners, and the development process.</p>
 
+<h4 id="ptransform-style-guidecontributeptransform-style-guide"><a href="/contribute/ptransform-style-guide">PTransform Style Guide</a></h4>
+<p>Learn about the best practices for developing new PTransforms.</p>
+
 <h4 id="resources">Resources</h4>
 <p>Resources for contributing to Beam, including mailing lists, the main GitHub repo, and the issues tracker.</p>
 

http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/contribute/logos/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/logos/index.html b/content/contribute/logos/index.html
index 96b1fd5..6ed790e 100644
--- a/content/contribute/logos/index.html
+++ b/content/contribute/logos/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/maturity-model/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/maturity-model/index.html b/content/contribute/maturity-model/index.html
index 1e1f174..9cbf5c5 100644
--- a/content/contribute/maturity-model/index.html
+++ b/content/contribute/maturity-model/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/presentation-materials/index.html
----------------------------------------------------------------------
diff --git a/content/contribute/presentation-materials/index.html b/content/contribute/presentation-materials/index.html
index d2fa7ea..ab19c76 100644
--- a/content/contribute/presentation-materials/index.html
+++ b/content/contribute/presentation-materials/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>


[3/5] beam-site git commit: Regenerate website

Posted by da...@apache.org.
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\u2019s</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\u2019 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\u2019s code, which others might want to reuse in ways you haven\u2019t 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\u2019s 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 \u201cIO\u201d: <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\u2019s 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\u2019s 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\u2019s 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 \u201cdelivery guarantee\u201d 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\u2019s 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\u2019s 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\u2019s 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\u2019t 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\u2019s 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\u2019re 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\u2019t be done.</li>
+</ul>
+
+<p>Do not:</p>
+
+<ul>
+  <li>If you can\u2019t handle a failure, don\u2019t even catch it.
+*Exception: *It may be valuable to catch the error, log a message, and rethrow it, if you\u2019re able to provide valuable context that the original error doesn\u2019t 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\u2019t 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\u2019s \u201cfusion\u201d, Apex \u201ccompute locality\u201d), but limit parallelization if these assumptions are violated. In that case you may need a \u201cfusion break\u201d (<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\u2019s 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\u2019s 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\u2019s \u201csuppressed exceptions\u201d). 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\u2019s hard to check if the output is correct, you can\u2019t attach a debugger, and you often can\u2019t 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\u2019t fit in memory, including \u201chot keys\u201d 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">\u201cfake\u201d</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\u2019s 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\u2019s 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\u2019re merely changing implementation or API - do not change API of the transform in a way that will make a user\u2019s 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\u2019s</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\u2019s not possible (e.g. JDBC does not provide a Beam-compatible (encodable with a Coder) \u201cJDBC record\u201d 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\u2019t 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 \u201cPackaging a family of transforms\u201d) 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\u2019s 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">\u2026</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">\u2026</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">\u2026</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">\u2026</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\u2019s 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\u2019s 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\u2019t 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\u2019s 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\u2019t want serialized as <code class="highlighter-rouge">transient</code>. Make classes <code class="highlighter-rouge">static</code> whenever possible (so that the instance doesn\u2019t 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>\u2019s <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>


[5/5] beam-site git commit: This closes #134

Posted by da...@apache.org.
This closes #134


Project: http://git-wip-us.apache.org/repos/asf/beam-site/repo
Commit: http://git-wip-us.apache.org/repos/asf/beam-site/commit/2f4d86d3
Tree: http://git-wip-us.apache.org/repos/asf/beam-site/tree/2f4d86d3
Diff: http://git-wip-us.apache.org/repos/asf/beam-site/diff/2f4d86d3

Branch: refs/heads/asf-site
Commit: 2f4d86d317b7fac2551c92a13805b99952ad9531
Parents: fedd4ab 077a174
Author: Davor Bonaci <da...@google.com>
Authored: Mon Jan 30 17:12:47 2017 -0800
Committer: Davor Bonaci <da...@google.com>
Committed: Mon Jan 30 17:12:47 2017 -0800

----------------------------------------------------------------------
 .../2016/03/17/capability-matrix.html           |   1 +
 .../2016/04/03/presentation-materials.html      |   1 +
 .../sdk/2016/02/25/python-sdk-now-public.html   |   1 +
 .../beam/release/2016/06/15/first-release.html  |   1 +
 .../10/11/strata-hadoop-world-and-beam.html     |   1 +
 .../website/2016/02/22/beam-has-a-logo.html     |   1 +
 .../blog/2016/05/18/splitAtFraction-method.html |   1 +
 .../05/27/where-is-my-pcollection-dot-map.html  |   1 +
 .../06/13/flink-batch-runner-milestone.html     |   1 +
 content/blog/2016/08/03/six-months.html         |   1 +
 content/blog/2016/10/20/test-stream.html        |   1 +
 content/blog/2017/01/09/added-apex-runner.html  |   1 +
 content/blog/2017/01/10/beam-graduates.html     |   1 +
 content/blog/index.html                         |   1 +
 content/coming-soon.html                        |   1 +
 .../contribute/contribution-guide/index.html    |   3 +
 content/contribute/design-principles/index.html |   1 +
 content/contribute/index.html                   |   4 +
 content/contribute/logos/index.html             |   1 +
 content/contribute/maturity-model/index.html    |   1 +
 .../presentation-materials/index.html           |   1 +
 .../ptransform-style-guide/index.html           | 759 +++++++++++++++++++
 content/contribute/release-guide/index.html     |   1 +
 content/contribute/source-repository/index.html |   1 +
 content/contribute/team/index.html              |   1 +
 content/contribute/testing/index.html           |   1 +
 content/contribute/work-in-progress/index.html  |   1 +
 content/documentation/index.html                |   1 +
 .../pipelines/create-your-pipeline/index.html   |   1 +
 .../pipelines/design-your-pipeline/index.html   |   1 +
 .../pipelines/test-your-pipeline/index.html     |   1 +
 .../documentation/programming-guide/index.html  |   1 +
 content/documentation/resources/index.html      |   1 +
 content/documentation/runners/apex/index.html   |   1 +
 .../runners/capability-matrix/index.html        |   1 +
 .../documentation/runners/dataflow/index.html   |   1 +
 content/documentation/runners/direct/index.html |   1 +
 content/documentation/runners/flink/index.html  |   1 +
 content/documentation/runners/spark/index.html  |   1 +
 content/documentation/sdks/java/index.html      |   1 +
 content/documentation/sdks/python/index.html    |   1 +
 content/get-started/beam-overview/index.html    |   1 +
 content/get-started/downloads/index.html        |   1 +
 content/get-started/index.html                  |   1 +
 .../mobile-gaming-example/index.html            |   1 +
 content/get-started/quickstart/index.html       |   1 +
 content/get-started/support/index.html          |   1 +
 .../get-started/wordcount-example/index.html    |   1 +
 content/index.html                              |   1 +
 content/privacy_policy/index.html               |   1 +
 src/_includes/header.html                       |   1 +
 src/contribute/contribution-guide.md            |   2 +
 src/contribute/index.md                         |   2 +
 src/contribute/ptransform-style-guide.md        | 489 ++++++++++++
 54 files changed, 1307 insertions(+)
----------------------------------------------------------------------



[2/5] beam-site git commit: Regenerate website

Posted by da...@apache.org.
http://git-wip-us.apache.org/repos/asf/beam-site/blob/077a1747/content/get-started/downloads/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/downloads/index.html b/content/get-started/downloads/index.html
index ed6a9f3..9dc7978 100644
--- a/content/get-started/downloads/index.html
+++ b/content/get-started/downloads/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/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/index.html b/content/get-started/index.html
index 4d6644d..d05f179 100644
--- a/content/get-started/index.html
+++ b/content/get-started/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/mobile-gaming-example/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/mobile-gaming-example/index.html b/content/get-started/mobile-gaming-example/index.html
index 4a94af5..1506502 100644
--- a/content/get-started/mobile-gaming-example/index.html
+++ b/content/get-started/mobile-gaming-example/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/quickstart/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/quickstart/index.html b/content/get-started/quickstart/index.html
index a11aa7b..53e0147 100644
--- a/content/get-started/quickstart/index.html
+++ b/content/get-started/quickstart/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/support/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/support/index.html b/content/get-started/support/index.html
index f191850..d5c9a09 100644
--- a/content/get-started/support/index.html
+++ b/content/get-started/support/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/wordcount-example/index.html
----------------------------------------------------------------------
diff --git a/content/get-started/wordcount-example/index.html b/content/get-started/wordcount-example/index.html
index 9c4e2b2..1c9820b 100644
--- a/content/get-started/wordcount-example/index.html
+++ b/content/get-started/wordcount-example/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/index.html
----------------------------------------------------------------------
diff --git a/content/index.html b/content/index.html
index bb58eb9..4ff3412 100644
--- a/content/index.html
+++ b/content/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/privacy_policy/index.html
----------------------------------------------------------------------
diff --git a/content/privacy_policy/index.html b/content/privacy_policy/index.html
index 97b6f2b..bf0a19f 100644
--- a/content/privacy_policy/index.html
+++ b/content/privacy_policy/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>