blob: 0c7583e09cdd7e0c087ff6055a2bfb4ec9c2cdc4 [file] [log] [blame]
<!DOCTYPE HTML PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
<!-- NewPage -->
<html lang="en">
<head>
<!-- Generated by javadoc (1.8.0_181-google-v7) on Mon Jan 27 16:42:30 PST 2020 -->
<title>ParDo (Apache Beam 2.20.0-SNAPSHOT)</title>
<meta name="date" content="2020-01-27">
<link rel="stylesheet" type="text/css" href="../../../../../stylesheet.css" title="Style">
<script type="text/javascript" src="../../../../../script.js"></script>
</head>
<body>
<script type="text/javascript"><!--
try {
if (location.href.indexOf('is-external=true') == -1) {
parent.document.title="ParDo (Apache Beam 2.20.0-SNAPSHOT)";
}
}
catch(err) {
}
//-->
var methods = {"i0":9,"i1":9};
var tabs = {65535:["t0","All Methods"],1:["t1","Static Methods"],8:["t4","Concrete Methods"]};
var altColor = "altColor";
var rowColor = "rowColor";
var tableTab = "tableTab";
var activeTableTab = "activeTableTab";
</script>
<noscript>
<div>JavaScript is disabled on your browser.</div>
</noscript>
<!-- ========= START OF TOP NAVBAR ======= -->
<div class="topNav"><a name="navbar.top">
<!-- -->
</a>
<div class="skipNav"><a href="#skip.navbar.top" title="Skip navigation links">Skip navigation links</a></div>
<a name="navbar.top.firstrow">
<!-- -->
</a>
<ul class="navList" title="Navigation">
<li><a href="../../../../../overview-summary.html">Overview</a></li>
<li><a href="package-summary.html">Package</a></li>
<li class="navBarCell1Rev">Class</li>
<li><a href="package-tree.html">Tree</a></li>
<li><a href="../../../../../deprecated-list.html">Deprecated</a></li>
<li><a href="../../../../../index-all.html">Index</a></li>
<li><a href="../../../../../help-doc.html">Help</a></li>
</ul>
</div>
<div class="subNav">
<ul class="navList">
<li><a href="../../../../../org/apache/beam/sdk/transforms/Min.html" title="class in org.apache.beam.sdk.transforms"><span class="typeNameLink">Prev&nbsp;Class</span></a></li>
<li><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.MultiOutput.html" title="class in org.apache.beam.sdk.transforms"><span class="typeNameLink">Next&nbsp;Class</span></a></li>
</ul>
<ul class="navList">
<li><a href="../../../../../index.html?org/apache/beam/sdk/transforms/ParDo.html" target="_top">Frames</a></li>
<li><a href="ParDo.html" target="_top">No&nbsp;Frames</a></li>
</ul>
<ul class="navList" id="allclasses_navbar_top">
<li><a href="../../../../../allclasses-noframe.html">All&nbsp;Classes</a></li>
</ul>
<div>
<script type="text/javascript"><!--
allClassesLink = document.getElementById("allclasses_navbar_top");
if(window==top) {
allClassesLink.style.display = "block";
}
else {
allClassesLink.style.display = "none";
}
//-->
</script>
</div>
<div>
<ul class="subNavList">
<li>Summary:&nbsp;</li>
<li><a href="#nested.class.summary">Nested</a>&nbsp;|&nbsp;</li>
<li>Field&nbsp;|&nbsp;</li>
<li><a href="#constructor.summary">Constr</a>&nbsp;|&nbsp;</li>
<li><a href="#method.summary">Method</a></li>
</ul>
<ul class="subNavList">
<li>Detail:&nbsp;</li>
<li>Field&nbsp;|&nbsp;</li>
<li><a href="#constructor.detail">Constr</a>&nbsp;|&nbsp;</li>
<li><a href="#method.detail">Method</a></li>
</ul>
</div>
<a name="skip.navbar.top">
<!-- -->
</a></div>
<!-- ========= END OF TOP NAVBAR ========= -->
<!-- ======== START OF CLASS DATA ======== -->
<div class="header">
<div class="subTitle">org.apache.beam.sdk.transforms</div>
<h2 title="Class ParDo" class="title">Class ParDo</h2>
</div>
<div class="contentContainer">
<ul class="inheritance">
<li>java.lang.Object</li>
<li>
<ul class="inheritance">
<li>org.apache.beam.sdk.transforms.ParDo</li>
</ul>
</li>
</ul>
<div class="description">
<ul class="blockList">
<li class="blockList">
<hr>
<br>
<pre>public class <span class="typeNameLabel">ParDo</span>
extends java.lang.Object</pre>
<div class="block"><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> is the core element-wise transform in Apache Beam, invoking a user-specified
function on each of the elements of the input <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> to produce zero or more output
elements, all of which are collected into the output <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>.
<p>Elements are processed independently, and possibly in parallel across distributed cloud
resources.
<p>The <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> processing style is similar to what happens inside the "Mapper" or "Reducer"
class of a MapReduce-style algorithm.
<h2><a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFns</code></a></h2>
<p>The function to use to process each element is specified by a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn&lt;InputT,
OutputT&gt;</code></a>, primarily via its <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.ProcessElement.html" title="annotation in org.apache.beam.sdk.transforms"><code>ProcessElement</code></a> method. The <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> may also provide a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.StartBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>StartBundle</code></a> and <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.FinishBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>finishBundle</code></a> method.
<p>Conceptually, when a <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> transform is executed, the elements of the input <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> are first divided up into some number of "bundles". These are farmed off to
distributed worker machines (or run locally, if using the <code>DirectRunner</code>). For each bundle
of input elements processing proceeds as follows:
<ol>
<li>If required, a fresh instance of the argument <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> is created on a worker, and the
<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.Setup.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.Setup</code></a> method is called on this instance. This may be through deserialization
or other means. A <a href="../../../../../org/apache/beam/sdk/PipelineRunner.html" title="class in org.apache.beam.sdk"><code>PipelineRunner</code></a> may reuse <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instances for multiple
bundles. A <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> that has terminated abnormally (by throwing an <code>Exception</code>)
will never be reused.
<li>The <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn's</code></a> <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.StartBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.StartBundle</code></a> method, if provided, is called to
initialize it.
<li>The <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn's</code></a> <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.ProcessElement.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.ProcessElement</code></a> method is called on each of the input
elements in the bundle.
<li>The <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn's</code></a> <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.FinishBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.FinishBundle</code></a> method, if provided, is called to
complete its work. After <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.FinishBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.FinishBundle</code></a> is called, the framework will not again
invoke <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.ProcessElement.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.ProcessElement</code></a> or <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.FinishBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.FinishBundle</code></a> until a new call to <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.StartBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.StartBundle</code></a> has occurred.
<li>If any of <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.Setup.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.Setup</code></a>, <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.StartBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.StartBundle</code></a>, <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.ProcessElement.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.ProcessElement</code></a> or
<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.FinishBundle.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.FinishBundle</code></a> methods throw an exception, the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.Teardown.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.Teardown</code></a> method, if
provided, will be called on the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instance.
<li>If a runner will no longer use a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a>, the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.Teardown.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.Teardown</code></a> method, if
provided, will be called on the discarded instance.
<li>If a bundle requested bundle finalization by registering a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.BundleFinalizer.Callback.html" title="interface in org.apache.beam.sdk.transforms"><code>bundle finalization callback</code></a>, the callback will be invoked
after the runner has successfully committed the output of a successful bundle.
</ol>
<p>Note also that calls to <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.Teardown.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.Teardown</code></a> are best effort, and may not be called before a
<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> is discarded in the general case. As a result, use of the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.Teardown.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.Teardown</code></a>
method to perform side effects is not appropriate, because the elements that produced the side
effect will not be replayed in case of failure, and those side effects are permanently lost.
<p>Each of the calls to any of the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn's</code></a> processing methods can produce zero or
more output elements. All of the of output elements from all of the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instances are
included in an output <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>.
<p>For example:
<pre><code> PCollection&lt;String&gt; lines = ...;
PCollection&lt;String&gt; words =
lines.apply(ParDo.of(new DoFn&lt;String, String&gt;() </code>{
<code> @ProcessElement
public void processElement(@Element String line,
OutputReceiver&lt;String&gt; r) {
for (String word : line.split("[^a-zA-Z']+")) {
r.output(word);
}
}</code>}));
<code> PCollection&lt;Integer&gt; wordLengths =
words.apply(ParDo.of(new DoFn&lt;String, Integer&gt;() </code>{
<code> @ProcessElement
public void processElement(@Element String word,
OutputReceiver&lt;Integer&gt; r) {
Integer length = word.length();
r.output(length);
}</code>}));
</pre>
<p>Each output element has the same timestamp and is in the same windows as its corresponding
input element, and the output <code>PCollection</code> has the same <a href="../../../../../org/apache/beam/sdk/transforms/windowing/WindowFn.html" title="class in org.apache.beam.sdk.transforms.windowing"><code>WindowFn</code></a> associated with
it as the input.
<h2>Naming <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> transforms</h2>
<p>The name of a transform is used to provide a name for any node in the <a href="../../../../../org/apache/beam/sdk/Pipeline.html" title="class in org.apache.beam.sdk"><code>Pipeline</code></a> graph
resulting from application of the transform. It is best practice to provide a name at the time of
application, via <a href="../../../../../org/apache/beam/sdk/values/PCollection.html#apply-java.lang.String-org.apache.beam.sdk.transforms.PTransform-"><code>PCollection.apply(String, PTransform)</code></a>. Otherwise, a unique name - which
may not be stable across pipeline revision - will be generated, based on the transform name.
<p>For example:
<pre><code> PCollection&lt;String&gt; words =
lines.apply("ExtractWords", ParDo.of(new DoFn&lt;String, String&gt;() { ... }));
PCollection&lt;Integer&gt; wordLengths =
words.apply("ComputeWordLengths", ParDo.of(new DoFn&lt;String, Integer&gt;() { ... }));
</code></pre>
<h2>Side Inputs</h2>
<p>While a <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> processes elements from a single "main input" <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>, it
can take additional "side input" <a href="../../../../../org/apache/beam/sdk/values/PCollectionView.html" title="interface in org.apache.beam.sdk.values"><code>PCollectionViews</code></a>. These side input
<a href="../../../../../org/apache/beam/sdk/values/PCollectionView.html" title="interface in org.apache.beam.sdk.values"><code>PCollectionViews</code></a> express styles of accessing <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollections</code></a> computed by earlier pipeline operations, passed in to the <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> transform
using <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.SingleOutput.html#withSideInputs-org.apache.beam.sdk.values.PCollectionView...-"><code>ParDo.SingleOutput.withSideInputs(org.apache.beam.sdk.values.PCollectionView&lt;?&gt;...)</code></a>, and their contents accessible to each of the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> operations via <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.ProcessContext.html#sideInput-org.apache.beam.sdk.values.PCollectionView-"><code>sideInput</code></a>. For example:
<pre><code> PCollection&lt;String&gt; words = ...;
PCollection&lt;Integer&gt; maxWordLengthCutOff = ...; // Singleton PCollection
final PCollectionView&lt;Integer&gt; maxWordLengthCutOffView =
maxWordLengthCutOff.apply(View.&lt;Integer&gt;asSingleton());
PCollection&lt;String&gt; wordsBelowCutOff =
words.apply(ParDo.of(new DoFn&lt;String, String&gt;() </code>{
<code> @ProcessElement
public void processElement(ProcessContext c) {
String word = c.element();
int lengthCutOff = c.sideInput(maxWordLengthCutOffView);
if (word.length() &lt;= lengthCutOff) {
c.output(word);
}
}</code>}).withSideInputs(maxWordLengthCutOffView));
</pre>
<h2>Additional Outputs</h2>
<p>Optionally, a <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> transform can produce multiple output <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollections</code></a>, both a "main output" <code>PCollection&lt;OutputT&gt;</code> plus any number of additional
output <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollections</code></a>, each keyed by a distinct <a href="../../../../../org/apache/beam/sdk/values/TupleTag.html" title="class in org.apache.beam.sdk.values"><code>TupleTag</code></a>, and bundled
in a <a href="../../../../../org/apache/beam/sdk/values/PCollectionTuple.html" title="class in org.apache.beam.sdk.values"><code>PCollectionTuple</code></a>. The <a href="../../../../../org/apache/beam/sdk/values/TupleTag.html" title="class in org.apache.beam.sdk.values"><code>TupleTags</code></a> to be used for the output <a href="../../../../../org/apache/beam/sdk/values/PCollectionTuple.html" title="class in org.apache.beam.sdk.values"><code>PCollectionTuple</code></a> are specified by invoking <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.SingleOutput.html#withOutputTags-org.apache.beam.sdk.values.TupleTag-org.apache.beam.sdk.values.TupleTagList-"><code>ParDo.SingleOutput.withOutputTags(org.apache.beam.sdk.values.TupleTag&lt;OutputT&gt;, org.apache.beam.sdk.values.TupleTagList)</code></a>. Unconsumed
outputs do not necessarily need to be explicitly specified, even if the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> generates
them. Within the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a>, an element is added to the main output <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> as
normal, using <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.WindowedContext.html#output-OutputT-"><code>DoFn.WindowedContext.output(Object)</code></a>, while an element is added to any additional
output <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> using <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.WindowedContext.html#output-org.apache.beam.sdk.values.TupleTag-T-"><code>DoFn.WindowedContext.output(TupleTag, Object)</code></a>. For example:
<pre><code> PCollection&lt;String&gt; words = ...;
// Select words whose length is below a cut off,
// plus the lengths of words that are above the cut off.
// Also select words starting with "MARKER".
final int wordLengthCutOff = 10;
// Create tags to use for the main and additional outputs.
final TupleTag&lt;String&gt; wordsBelowCutOffTag =
new TupleTag&lt;String&gt;(){};
final TupleTag&lt;Integer&gt; wordLengthsAboveCutOffTag =
new TupleTag&lt;Integer&gt;(){};
final TupleTag&lt;String&gt; markedWordsTag =
new TupleTag&lt;String&gt;(){};
PCollectionTuple results =
words.apply(
ParDo
.of(new DoFn&lt;String, String&gt;() {
// Create a tag for the unconsumed output.
final TupleTag&lt;String&gt; specialWordsTag =
new TupleTag&lt;String&gt;(){};}</code>
<code> @ProcessElement
public void processElement(@Element String word, MultiOutputReceiver r) {
if (word.length() &lt;= wordLengthCutOff) {
// Emit this short word to the main output.
r.get(wordsBelowCutOffTag).output(word);
} else {
// Emit this long word's length to a specified output.
r.get(wordLengthsAboveCutOffTag).output(word.length());
}
if (word.startsWith("MARKER")) {
// Emit this word to a different specified output.
r.get(markedWordsTag).output(word);
}
if (word.startsWith("SPECIAL")) {
// Emit this word to the unconsumed output.
r.get(specialWordsTag).output(word);
}
}</code>})
// Specify the main and consumed output tags of the
// PCollectionTuple result:
.withOutputTags(wordsBelowCutOffTag,
TupleTagList.of(wordLengthsAboveCutOffTag)
.and(markedWordsTag)));
// Extract the PCollection results, by tag.<code>
PCollection&lt;String&gt; wordsBelowCutOff =
results.get(wordsBelowCutOffTag);
PCollection&lt;Integer&gt; wordLengthsAboveCutOff =
results.get(wordLengthsAboveCutOffTag);
PCollection&lt;String&gt; markedWords =
results.get(markedWordsTag);
</code></pre>
<h2>Output Coders</h2>
<p>By default, the <a href="../../../../../org/apache/beam/sdk/coders/Coder.html" title="class in org.apache.beam.sdk.coders"><code>Coder&lt;OutputT&gt;</code></a> for the elements of the main output <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection&lt;OutputT&gt;</code></a> is inferred from the concrete type of the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn&lt;InputT, OutputT&gt;</code></a>.
<p>By default, the <a href="../../../../../org/apache/beam/sdk/coders/Coder.html" title="class in org.apache.beam.sdk.coders"><code>Coder&lt;AdditionalOutputT&gt;</code></a> for the elements of an output
<a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection&lt;AdditionalOutputT&gt;</code></a> is inferred from the concrete type of
the corresponding <a href="../../../../../org/apache/beam/sdk/values/TupleTag.html" title="class in org.apache.beam.sdk.values"><code>TupleTag&lt;AdditionalOutputT&gt;</code></a>. To be successful, the
<a href="../../../../../org/apache/beam/sdk/values/TupleTag.html" title="class in org.apache.beam.sdk.values"><code>TupleTag</code></a> should be created as an instance of a trivial anonymous subclass, with <code>{}</code> suffixed to the constructor call. Such uses block Java's generic type parameter inference, so
the <code>&lt;X&gt;</code> argument must be provided explicitly. For example:
<pre><code>
// A TupleTag to use for a side input can be written concisely:
final TupleTag&lt;Integer&gt; sideInputag = new TupleTag&lt;&gt;();
// A TupleTag to use for an output should be written with "{}",
// and explicit generic parameter type:
final TupleTag&lt;String&gt; additionalOutputTag = new TupleTag&lt;String&gt;(){};
</code></pre>
This style of <code>TupleTag</code> instantiation is used in the example of <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDos</code></a> that
produce multiple outputs, above.
<h2>Serializability of <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFns</code></a></h2>
<p>A <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> passed to a <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> transform must be <code>Serializable</code>. This allows
the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instance created in this "main program" to be sent (in serialized form) to
remote worker machines and reconstituted for bundles of elements of the input <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>
being processed. A <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> can have instance variable state, and non-transient instance
variable state will be serialized in the main program and then deserialized on remote worker
machines for some number of bundles of elements to process.
<p><a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFns</code></a> expressed as anonymous inner classes can be convenient, but due to a quirk
in Java's rules for serializability, non-static inner or nested classes (including anonymous
inner classes) automatically capture their enclosing class's instance in their serialized state.
This can lead to including much more than intended in the serialized state of a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a>, or
even things that aren't <code>Serializable</code>.
<p>There are two ways to avoid unintended serialized state in a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a>:
<ul>
<li>Define the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> as a named, static class.
<li>Define the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> as an anonymous inner class inside of a static method.
</ul>
<p>Both of these approaches ensure that there is no implicit enclosing instance serialized along
with the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instance.
<p>Prior to Java 8, any local variables of the enclosing method referenced from within an
anonymous inner class need to be marked as <code>final</code>. If defining the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> as a named
static class, such variables would be passed as explicit constructor arguments and stored in
explicit instance variables.
<p>There are three main ways to initialize the state of a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instance processing a
bundle:
<ul>
<li>Define instance variable state (including implicit instance variables holding final
variables captured by an anonymous inner class), initialized by the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a>'s
constructor (which is implicit for an anonymous inner class). This state will be
automatically serialized and then deserialized in the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instances created for
bundles. This method is good for state known when the original <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> is created in
the main program, if it's not overly large. This is not suitable for any state which must
only be used for a single bundle, as <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn's</code></a> may be used to process multiple
bundles.
<li>Compute the state as a singleton <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> and pass it in as a side input to the
<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a>. This is good if the state needs to be computed by the pipeline, or if the
state is very large and so is best read from file(s) rather than sent as part of the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn's</code></a> serialized state.
<li>Initialize the state in each <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instance, in a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.Setup.html" title="annotation in org.apache.beam.sdk.transforms"><code>DoFn.Setup</code></a> method. This is
good if the initialization doesn't depend on any information known only by the main program
or computed by earlier pipeline operations, but is the same for all instances of this
<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> for all program executions, say setting up empty caches or initializing
constant data.
</ul>
<h2>No Global Shared State</h2>
<p><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> operations are intended to be able to run in parallel across multiple worker
machines. This precludes easy sharing and updating mutable state across those machines. There is
no support in the Beam model for communicating and synchronizing updates to shared state across
worker machines, so programs should not access any mutable static variable state in their <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a>, without understanding that the Java processes for the main program and workers will each
have its own independent copy of such state, and there won't be any automatic copying of that
state across Java processes. All information should be communicated to <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instances via
main and side inputs and serialized state, and all output should be communicated from a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instance via output <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollections</code></a>, in the absence of external
communication mechanisms written by user code.
<h2>Fault Tolerance</h2>
<p>In a distributed system, things can fail: machines can crash, machines can be unable to
communicate across the network, etc. While individual failures are rare, the larger the job, the
greater the chance that something, somewhere, will fail. Beam runners may strive to mask such
failures by retrying failed <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> bundle. This means that a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instance might
process a bundle partially, then crash for some reason, then be rerun (often in a new JVM) on
that same bundle and on the same elements as before. Sometimes two or more <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> instances
will be running on the same bundle simultaneously, with the system taking the results of the
first instance to complete successfully. Consequently, the code in a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> needs to be
written such that these duplicate (sequential or concurrent) executions do not cause problems. If
the outputs of a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> are a pure function of its inputs, then this requirement is
satisfied. However, if a <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn's</code></a> execution has external side-effects, such as
performing updates to external HTTP services, then the <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn's</code></a> code needs to take
care to ensure that those updates are idempotent and that concurrent updates are acceptable. This
property can be difficult to achieve, so it is advisable to strive to keep <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFns</code></a> as
pure functions as much as possible.
<h2>Optimization</h2>
<p>Beam runners may choose to apply optimizations to a pipeline before it is executed. A key
optimization, <i>fusion</i>, relates to <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> operations. If one <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> operation
produces a <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> that is then consumed as the main input of another <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a>
operation, the two <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> operations will be <i>fused</i> together into a single ParDo
operation and run in a single pass; this is "producer-consumer fusion". Similarly, if two or more
ParDo operations have the same <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> main input, they will be fused into a single
<a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> that makes just one pass over the input <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a>; this is "sibling
fusion".
<p>If after fusion there are no more unfused references to a <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> (e.g., one
between a producer ParDo and a consumer <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a>), the <a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values"><code>PCollection</code></a> itself is "fused
away" and won't ever be written to disk, saving all the I/O and space expense of constructing it.
<p>When Beam runners apply fusion optimization, it is essentially "free" to write <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a>
operations in a very modular, composable style, each <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> operation doing one clear
task, and stringing together sequences of <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> operations to get the desired overall
effect. Such programs can be easier to understand, easier to unit-test, easier to extend and
evolve, and easier to reuse in new programs. The predefined library of PTransforms that come with
Beam makes heavy use of this modular, composable style, trusting to the runner to "flatten out"
all the compositions into highly optimized stages.</div>
<dl>
<dt><span class="seeLabel">See Also:</span></dt>
<dd><a href= "https://beam.apache.org/documentation/programming-guide/#pardo"> the web
documentation for ParDo</a></dd>
</dl>
</li>
</ul>
</div>
<div class="summary">
<ul class="blockList">
<li class="blockList">
<!-- ======== NESTED CLASS SUMMARY ======== -->
<ul class="blockList">
<li class="blockList"><a name="nested.class.summary">
<!-- -->
</a>
<h3>Nested Class Summary</h3>
<table class="memberSummary" border="0" cellpadding="3" cellspacing="0" summary="Nested Class Summary table, listing nested classes, and an explanation">
<caption><span>Nested Classes</span><span class="tabEnd">&nbsp;</span></caption>
<tr>
<th class="colFirst" scope="col">Modifier and Type</th>
<th class="colLast" scope="col">Class and Description</th>
</tr>
<tr class="altColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.MultiOutput.html" title="class in org.apache.beam.sdk.transforms">ParDo.MultiOutput</a>&lt;<a href="../../../../../org/apache/beam/sdk/transforms/ParDo.MultiOutput.html" title="type parameter in ParDo.MultiOutput">InputT</a>,<a href="../../../../../org/apache/beam/sdk/transforms/ParDo.MultiOutput.html" title="type parameter in ParDo.MultiOutput">OutputT</a>&gt;</span></code>
<div class="block">A <a href="../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> that, when applied to a <code>PCollection&lt;InputT&gt;</code>, invokes a
user-specified <code>DoFn&lt;InputT, OutputT&gt;</code> on all its elements, which can emit elements to
any of the <a href="../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a>'s output <code>PCollection</code>s, which are bundled into a result
<code>PCollectionTuple</code>.</div>
</td>
</tr>
<tr class="rowColor">
<td class="colFirst"><code>static class&nbsp;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.SingleOutput.html" title="class in org.apache.beam.sdk.transforms">ParDo.SingleOutput</a>&lt;<a href="../../../../../org/apache/beam/sdk/transforms/ParDo.SingleOutput.html" title="type parameter in ParDo.SingleOutput">InputT</a>,<a href="../../../../../org/apache/beam/sdk/transforms/ParDo.SingleOutput.html" title="type parameter in ParDo.SingleOutput">OutputT</a>&gt;</span></code>
<div class="block">A <a href="../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> that, when applied to a <code>PCollection&lt;InputT&gt;</code>, invokes a
user-specified <code>DoFn&lt;InputT, OutputT&gt;</code> on all its elements, with all its outputs
collected into an output <code>PCollection&lt;OutputT&gt;</code>.</div>
</td>
</tr>
</table>
</li>
</ul>
<!-- ======== CONSTRUCTOR SUMMARY ======== -->
<ul class="blockList">
<li class="blockList"><a name="constructor.summary">
<!-- -->
</a>
<h3>Constructor Summary</h3>
<table class="memberSummary" border="0" cellpadding="3" cellspacing="0" summary="Constructor Summary table, listing constructors, and an explanation">
<caption><span>Constructors</span><span class="tabEnd">&nbsp;</span></caption>
<tr>
<th class="colOne" scope="col">Constructor and Description</th>
</tr>
<tr class="altColor">
<td class="colOne"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html#ParDo--">ParDo</a></span>()</code>&nbsp;</td>
</tr>
</table>
</li>
</ul>
<!-- ========== METHOD SUMMARY =========== -->
<ul class="blockList">
<li class="blockList"><a name="method.summary">
<!-- -->
</a>
<h3>Method Summary</h3>
<table class="memberSummary" border="0" cellpadding="3" cellspacing="0" summary="Method Summary table, listing methods, and an explanation">
<caption><span id="t0" class="activeTableTab"><span>All Methods</span><span class="tabEnd">&nbsp;</span></span><span id="t1" class="tableTab"><span><a href="javascript:show(1);">Static Methods</a></span><span class="tabEnd">&nbsp;</span></span><span id="t4" class="tableTab"><span><a href="javascript:show(8);">Concrete Methods</a></span><span class="tabEnd">&nbsp;</span></span></caption>
<tr>
<th class="colFirst" scope="col">Modifier and Type</th>
<th class="colLast" scope="col">Method and Description</th>
</tr>
<tr id="i0" class="altColor">
<td class="colFirst"><code>static <a href="../../../../../org/apache/beam/sdk/transforms/DoFnSchemaInformation.html" title="class in org.apache.beam.sdk.transforms">DoFnSchemaInformation</a></code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html#getDoFnSchemaInformation-org.apache.beam.sdk.transforms.DoFn-org.apache.beam.sdk.values.PCollection-">getDoFnSchemaInformation</a></span>(<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms">DoFn</a>&lt;?,?&gt;&nbsp;fn,
<a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values">PCollection</a>&lt;?&gt;&nbsp;input)</code>
<div class="block">Extract information on how the DoFn uses schemas.</div>
</td>
</tr>
<tr id="i1" class="rowColor">
<td class="colFirst"><code>static &lt;InputT,OutputT&gt;<br><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.SingleOutput.html" title="class in org.apache.beam.sdk.transforms">ParDo.SingleOutput</a>&lt;InputT,OutputT&gt;</code></td>
<td class="colLast"><code><span class="memberNameLink"><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html#of-org.apache.beam.sdk.transforms.DoFn-">of</a></span>(<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms">DoFn</a>&lt;InputT,OutputT&gt;&nbsp;fn)</code>
<div class="block">Creates a <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> <a href="../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> that will invoke the given <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> function.</div>
</td>
</tr>
</table>
<ul class="blockList">
<li class="blockList"><a name="methods.inherited.from.class.java.lang.Object">
<!-- -->
</a>
<h3>Methods inherited from class&nbsp;java.lang.Object</h3>
<code>clone, equals, finalize, getClass, hashCode, notify, notifyAll, toString, wait, wait, wait</code></li>
</ul>
</li>
</ul>
</li>
</ul>
</div>
<div class="details">
<ul class="blockList">
<li class="blockList">
<!-- ========= CONSTRUCTOR DETAIL ======== -->
<ul class="blockList">
<li class="blockList"><a name="constructor.detail">
<!-- -->
</a>
<h3>Constructor Detail</h3>
<a name="ParDo--">
<!-- -->
</a>
<ul class="blockListLast">
<li class="blockList">
<h4>ParDo</h4>
<pre>public&nbsp;ParDo()</pre>
</li>
</ul>
</li>
</ul>
<!-- ============ METHOD DETAIL ========== -->
<ul class="blockList">
<li class="blockList"><a name="method.detail">
<!-- -->
</a>
<h3>Method Detail</h3>
<a name="of-org.apache.beam.sdk.transforms.DoFn-">
<!-- -->
</a>
<ul class="blockList">
<li class="blockList">
<h4>of</h4>
<pre>public static&nbsp;&lt;InputT,OutputT&gt;&nbsp;<a href="../../../../../org/apache/beam/sdk/transforms/ParDo.SingleOutput.html" title="class in org.apache.beam.sdk.transforms">ParDo.SingleOutput</a>&lt;InputT,OutputT&gt;&nbsp;of(<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms">DoFn</a>&lt;InputT,OutputT&gt;&nbsp;fn)</pre>
<div class="block">Creates a <a href="../../../../../org/apache/beam/sdk/transforms/ParDo.html" title="class in org.apache.beam.sdk.transforms"><code>ParDo</code></a> <a href="../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> that will invoke the given <a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms"><code>DoFn</code></a> function.
<p>The resulting <a href="../../../../../org/apache/beam/sdk/transforms/PTransform.html" title="class in org.apache.beam.sdk.transforms"><code>PTransform</code></a> is ready to be applied, or further properties
can be set on it first.</div>
</li>
</ul>
<a name="getDoFnSchemaInformation-org.apache.beam.sdk.transforms.DoFn-org.apache.beam.sdk.values.PCollection-">
<!-- -->
</a>
<ul class="blockListLast">
<li class="blockList">
<h4>getDoFnSchemaInformation</h4>
<pre><a href="../../../../../org/apache/beam/sdk/annotations/Internal.html" title="annotation in org.apache.beam.sdk.annotations">@Internal</a>
public static&nbsp;<a href="../../../../../org/apache/beam/sdk/transforms/DoFnSchemaInformation.html" title="class in org.apache.beam.sdk.transforms">DoFnSchemaInformation</a>&nbsp;getDoFnSchemaInformation(<a href="../../../../../org/apache/beam/sdk/transforms/DoFn.html" title="class in org.apache.beam.sdk.transforms">DoFn</a>&lt;?,?&gt;&nbsp;fn,
<a href="../../../../../org/apache/beam/sdk/values/PCollection.html" title="class in org.apache.beam.sdk.values">PCollection</a>&lt;?&gt;&nbsp;input)</pre>
<div class="block">Extract information on how the DoFn uses schemas. In particular, if the schema of an element
parameter does not match the input PCollection's schema, convert.</div>
</li>
</ul>
</li>
</ul>
</li>
</ul>
</div>
</div>
<!-- ========= END OF CLASS DATA ========= -->
<!-- ======= START OF BOTTOM NAVBAR ====== -->
<div class="bottomNav"><a name="navbar.bottom">
<!-- -->
</a>
<div class="skipNav"><a href="#skip.navbar.bottom" title="Skip navigation links">Skip navigation links</a></div>
<a name="navbar.bottom.firstrow">
<!-- -->
</a>
<ul class="navList" title="Navigation">
<li><a href="../../../../../overview-summary.html">Overview</a></li>
<li><a href="package-summary.html">Package</a></li>
<li class="navBarCell1Rev">Class</li>
<li><a href="package-tree.html">Tree</a></li>
<li><a href="../../../../../deprecated-list.html">Deprecated</a></li>
<li><a href="../../../../../index-all.html">Index</a></li>
<li><a href="../../../../../help-doc.html">Help</a></li>
</ul>
</div>
<div class="subNav">
<ul class="navList">
<li><a href="../../../../../org/apache/beam/sdk/transforms/Min.html" title="class in org.apache.beam.sdk.transforms"><span class="typeNameLink">Prev&nbsp;Class</span></a></li>
<li><a href="../../../../../org/apache/beam/sdk/transforms/ParDo.MultiOutput.html" title="class in org.apache.beam.sdk.transforms"><span class="typeNameLink">Next&nbsp;Class</span></a></li>
</ul>
<ul class="navList">
<li><a href="../../../../../index.html?org/apache/beam/sdk/transforms/ParDo.html" target="_top">Frames</a></li>
<li><a href="ParDo.html" target="_top">No&nbsp;Frames</a></li>
</ul>
<ul class="navList" id="allclasses_navbar_bottom">
<li><a href="../../../../../allclasses-noframe.html">All&nbsp;Classes</a></li>
</ul>
<div>
<script type="text/javascript"><!--
allClassesLink = document.getElementById("allclasses_navbar_bottom");
if(window==top) {
allClassesLink.style.display = "block";
}
else {
allClassesLink.style.display = "none";
}
//-->
</script>
</div>
<div>
<ul class="subNavList">
<li>Summary:&nbsp;</li>
<li><a href="#nested.class.summary">Nested</a>&nbsp;|&nbsp;</li>
<li>Field&nbsp;|&nbsp;</li>
<li><a href="#constructor.summary">Constr</a>&nbsp;|&nbsp;</li>
<li><a href="#method.summary">Method</a></li>
</ul>
<ul class="subNavList">
<li>Detail:&nbsp;</li>
<li>Field&nbsp;|&nbsp;</li>
<li><a href="#constructor.detail">Constr</a>&nbsp;|&nbsp;</li>
<li><a href="#method.detail">Method</a></li>
</ul>
</div>
<a name="skip.navbar.bottom">
<!-- -->
</a></div>
<!-- ======== END OF BOTTOM NAVBAR ======= -->
</body>
</html>