blob: b8e5069abd0aba8d61332103f49fd1168e13df94 [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.transforms.util module &mdash; Apache Beam documentation</title>
<link rel="stylesheet" href="_static/css/theme.css" type="text/css" />
<link rel="index" title="Index"
href="genindex.html"/>
<link rel="search" title="Search" href="search.html"/>
<link rel="top" title="Apache Beam documentation" href="index.html"/>
<link rel="up" title="apache_beam.transforms package" href="apache_beam.transforms.html"/>
<link rel="next" title="apache_beam.transforms.window module" href="apache_beam.transforms.window.html"/>
<link rel="prev" title="apache_beam.transforms.userstate module" href="apache_beam.transforms.userstate.html"/>
<script src="_static/js/modernizr.min.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search">
<a href="index.html" class="icon icon-home"> Apache Beam
</a>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul class="current">
<li class="toctree-l1"><a class="reference internal" href="apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.internal.html">apache_beam.internal package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.tools.html">apache_beam.tools package</a></li>
<li class="toctree-l1 current"><a class="reference internal" href="apache_beam.transforms.html">apache_beam.transforms package</a><ul class="current">
<li class="toctree-l2 current"><a class="reference internal" href="apache_beam.transforms.html#submodules">Submodules</a><ul class="current">
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.combiners.html">apache_beam.transforms.combiners module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.core.html">apache_beam.transforms.core module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.create_source.html">apache_beam.transforms.create_source module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.display.html">apache_beam.transforms.display module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.external.html">apache_beam.transforms.external module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.external_test_it.html">apache_beam.transforms.external_test_it module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.ptransform.html">apache_beam.transforms.ptransform module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.sideinputs.html">apache_beam.transforms.sideinputs module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.stats.html">apache_beam.transforms.stats module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.timeutil.html">apache_beam.transforms.timeutil module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.trigger.html">apache_beam.transforms.trigger module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.userstate.html">apache_beam.transforms.userstate module</a></li>
<li class="toctree-l3 current"><a class="current reference internal" href="#">apache_beam.transforms.util module</a></li>
<li class="toctree-l3"><a class="reference internal" href="apache_beam.transforms.window.html">apache_beam.transforms.window module</a></li>
</ul>
</li>
</ul>
</li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.utils.html">apache_beam.utils package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
<li class="toctree-l1"><a class="reference internal" href="apache_beam.version.html">apache_beam.version module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="index.html">Docs</a> &raquo;</li>
<li><a href="apache_beam.transforms.html">apache_beam.transforms package</a> &raquo;</li>
<li>apache_beam.transforms.util module</li>
<li class="wy-breadcrumbs-aside">
<a href="_sources/apache_beam.transforms.util.rst.txt" rel="nofollow"> View page source</a>
</li>
</ul>
<hr/>
</div>
<div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article">
<div itemprop="articleBody">
<div class="section" id="module-apache_beam.transforms.util">
<span id="apache-beam-transforms-util-module"></span><h1>apache_beam.transforms.util module<a class="headerlink" href="#module-apache_beam.transforms.util" title="Permalink to this headline"></a></h1>
<p>Simple utility PTransforms.</p>
<dl class="class">
<dt id="apache_beam.transforms.util.CoGroupByKey">
<em class="property">class </em><code class="descclassname">apache_beam.transforms.util.</code><code class="descname">CoGroupByKey</code><span class="sig-paren">(</span><em>**kwargs</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#CoGroupByKey"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.CoGroupByKey" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>Groups results across several PCollections by key.</p>
<p>Given an input dict of serializable keys (called “tags”) to 0 or more
PCollections of (key, value) tuples, it creates a single output PCollection
of (key, value) tuples whose keys are the unique input keys from all inputs,
and whose values are dicts mapping each tag to an iterable of whatever values
were under the key in the corresponding PCollection, in this manner:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="p">(</span><span class="s1">&#39;some key&#39;</span><span class="p">,</span> <span class="p">{</span><span class="s1">&#39;tag1&#39;</span><span class="p">:</span> <span class="p">[</span><span class="s1">&#39;value 1 under &quot;some key&quot; in pcoll1&#39;</span><span class="p">,</span>
<span class="s1">&#39;value 2 under &quot;some key&quot; in pcoll1&#39;</span><span class="p">,</span>
<span class="o">...</span><span class="p">],</span>
<span class="s1">&#39;tag2&#39;</span><span class="p">:</span> <span class="o">...</span> <span class="p">,</span>
<span class="o">...</span> <span class="p">})</span>
</pre></div>
</div>
<p>For example, given:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="p">{</span><span class="s1">&#39;tag1&#39;</span><span class="p">:</span> <span class="n">pc1</span><span class="p">,</span> <span class="s1">&#39;tag2&#39;</span><span class="p">:</span> <span class="n">pc2</span><span class="p">,</span> <span class="mi">333</span><span class="p">:</span> <span class="n">pc3</span><span class="p">}</span>
</pre></div>
</div>
<p>where:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="n">pc1</span> <span class="o">=</span> <span class="p">[(</span><span class="n">k1</span><span class="p">,</span> <span class="n">v1</span><span class="p">)]</span>
<span class="n">pc2</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">pc3</span> <span class="o">=</span> <span class="p">[(</span><span class="n">k1</span><span class="p">,</span> <span class="n">v31</span><span class="p">),</span> <span class="p">(</span><span class="n">k1</span><span class="p">,</span> <span class="n">v32</span><span class="p">),</span> <span class="p">(</span><span class="n">k2</span><span class="p">,</span> <span class="n">v33</span><span class="p">)]</span>
</pre></div>
</div>
<p>The output PCollection would be:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="p">[(</span><span class="n">k1</span><span class="p">,</span> <span class="p">{</span><span class="s1">&#39;tag1&#39;</span><span class="p">:</span> <span class="p">[</span><span class="n">v1</span><span class="p">],</span> <span class="s1">&#39;tag2&#39;</span><span class="p">:</span> <span class="p">[],</span> <span class="mi">333</span><span class="p">:</span> <span class="p">[</span><span class="n">v31</span><span class="p">,</span> <span class="n">v32</span><span class="p">]}),</span>
<span class="p">(</span><span class="n">k2</span><span class="p">,</span> <span class="p">{</span><span class="s1">&#39;tag1&#39;</span><span class="p">:</span> <span class="p">[],</span> <span class="s1">&#39;tag2&#39;</span><span class="p">:</span> <span class="p">[],</span> <span class="mi">333</span><span class="p">:</span> <span class="p">[</span><span class="n">v33</span><span class="p">]})]</span>
</pre></div>
</div>
<p>CoGroupByKey also works for tuples, lists, or other flat iterables of
PCollections, in which case the values of the resulting PCollections
will be tuples whose nth value is the list of values from the nth
PCollection—conceptually, the “tags” are the indices into the input.
Thus, for this input:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="p">(</span><span class="n">pc1</span><span class="p">,</span> <span class="n">pc2</span><span class="p">,</span> <span class="n">pc3</span><span class="p">)</span>
</pre></div>
</div>
<p>the output would be:</p>
<div class="highlight-default"><div class="highlight"><pre><span></span><span class="p">[(</span><span class="n">k1</span><span class="p">,</span> <span class="p">([</span><span class="n">v1</span><span class="p">],</span> <span class="p">[],</span> <span class="p">[</span><span class="n">v31</span><span class="p">,</span> <span class="n">v32</span><span class="p">]),</span>
<span class="p">(</span><span class="n">k2</span><span class="p">,</span> <span class="p">([],</span> <span class="p">[],</span> <span class="p">[</span><span class="n">v33</span><span class="p">]))]</span>
</pre></div>
</div>
<dl class="attribute">
<dt>
<code class="descname">**kwargs</code></dt>
<dd><p>Accepts a single named argument “pipeline”, which specifies the
pipeline that “owns” this PTransform. Ordinarily CoGroupByKey can obtain
this information from one of the input PCollections, but if there are none
(or if there’s a chance there may be none), this argument is the only way
to provide pipeline information, and should be considered mandatory.</p>
</dd></dl>
<dl class="method">
<dt id="apache_beam.transforms.util.CoGroupByKey.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcolls</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#CoGroupByKey.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.CoGroupByKey.expand" title="Permalink to this definition"></a></dt>
<dd><p>Performs CoGroupByKey on argument pcolls; see class docstring.</p>
</dd></dl>
</dd></dl>
<dl class="function">
<dt id="apache_beam.transforms.util.Keys">
<code class="descclassname">apache_beam.transforms.util.</code><code class="descname">Keys</code><span class="sig-paren">(</span><em>label='Keys'</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Keys"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Keys" title="Permalink to this definition"></a></dt>
<dd><p>Produces a PCollection of first elements of 2-tuples in a PCollection.</p>
</dd></dl>
<dl class="function">
<dt id="apache_beam.transforms.util.Values">
<code class="descclassname">apache_beam.transforms.util.</code><code class="descname">Values</code><span class="sig-paren">(</span><em>label='Values'</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Values"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Values" title="Permalink to this definition"></a></dt>
<dd><p>Produces a PCollection of second elements of 2-tuples in a PCollection.</p>
</dd></dl>
<dl class="function">
<dt id="apache_beam.transforms.util.KvSwap">
<code class="descclassname">apache_beam.transforms.util.</code><code class="descname">KvSwap</code><span class="sig-paren">(</span><em>label='KvSwap'</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#KvSwap"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.KvSwap" title="Permalink to this definition"></a></dt>
<dd><p>Produces a PCollection reversing 2-tuples in a PCollection.</p>
</dd></dl>
<dl class="function">
<dt id="apache_beam.transforms.util.Distinct">
<code class="descclassname">apache_beam.transforms.util.</code><code class="descname">Distinct</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Distinct"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Distinct" title="Permalink to this definition"></a></dt>
<dd><p>Produces a PCollection containing distinct elements of a PCollection.</p>
</dd></dl>
<dl class="function">
<dt id="apache_beam.transforms.util.RemoveDuplicates">
<code class="descclassname">apache_beam.transforms.util.</code><code class="descname">RemoveDuplicates</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#RemoveDuplicates"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.RemoveDuplicates" title="Permalink to this definition"></a></dt>
<dd><p>Produces a PCollection containing distinct elements of a PCollection.</p>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.BatchElements">
<em class="property">class </em><code class="descclassname">apache_beam.transforms.util.</code><code class="descname">BatchElements</code><span class="sig-paren">(</span><em>min_batch_size=1</em>, <em>max_batch_size=10000</em>, <em>target_batch_overhead=0.05</em>, <em>target_batch_duration_secs=1</em>, <em>variance=0.25</em>, <em>clock=&lt;built-in function time&gt;</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#BatchElements"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.BatchElements" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>A Transform that batches elements for amortized processing.</p>
<p>This transform is designed to precede operations whose processing cost
is of the form</p>
<blockquote>
<div>time = fixed_cost + num_elements * per_element_cost</div></blockquote>
<p>where the per element cost is (often significantly) smaller than the fixed
cost and could be amortized over multiple elements. It consumes a PCollection
of element type T and produces a PCollection of element type List[T].</p>
<p>This transform attempts to find the best batch size between the minimim
and maximum parameters by profiling the time taken by (fused) downstream
operations. For a fixed batch size, set the min and max to be equal.</p>
<p>Elements are batched per-window and batches emitted in the window
corresponding to its contents.</p>
<table class="docutils field-list" frame="void" rules="none">
<col class="field-name" />
<col class="field-body" />
<tbody valign="top">
<tr class="field-odd field"><th class="field-name">Parameters:</th><td class="field-body"><ul class="first last simple">
<li><strong>min_batch_size</strong> – (optional) the smallest number of elements per batch</li>
<li><strong>max_batch_size</strong> – (optional) the largest number of elements per batch</li>
<li><strong>target_batch_overhead</strong> – (optional) a target for fixed_cost / time,
as used in the formula above</li>
<li><strong>target_batch_duration_secs</strong> – (optional) a target for total time per bundle,
in seconds</li>
<li><strong>variance</strong> – (optional) the permitted (relative) amount of deviation from the
(estimated) ideal batch size used to produce a wider base for
linear interpolation</li>
<li><strong>clock</strong> – (optional) an alternative to time.time for measuring the cost of
donwstream operations (mostly for testing)</li>
</ul>
</td>
</tr>
</tbody>
</table>
<dl class="method">
<dt id="apache_beam.transforms.util.BatchElements.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#BatchElements.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.BatchElements.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.Reshuffle">
<em class="property">class </em><code class="descclassname">apache_beam.transforms.util.</code><code class="descname">Reshuffle</code><span class="sig-paren">(</span><em>label=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reshuffle"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reshuffle" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>PTransform that returns a PCollection equivalent to its input,
but operationally provides some of the side effects of a GroupByKey,
in particular preventing fusion of the surrounding transforms,
checkpointing, and deduplication by id.</p>
<p>Reshuffle adds a temporary random key to each element, performs a
ReshufflePerKey, and finally removes the temporary key.</p>
<p>Reshuffle is experimental. No backwards compatibility guarantees.</p>
<dl class="method">
<dt id="apache_beam.transforms.util.Reshuffle.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reshuffle.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reshuffle.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="method">
<dt id="apache_beam.transforms.util.Reshuffle.to_runner_api_parameter">
<code class="descname">to_runner_api_parameter</code><span class="sig-paren">(</span><em>unused_context</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reshuffle.to_runner_api_parameter"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reshuffle.to_runner_api_parameter" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="staticmethod">
<dt id="apache_beam.transforms.util.Reshuffle.from_runner_api_parameter">
<em class="property">static </em><code class="descname">from_runner_api_parameter</code><span class="sig-paren">(</span><em>unused_parameter</em>, <em>unused_context</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reshuffle.from_runner_api_parameter"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reshuffle.from_runner_api_parameter" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="function">
<dt id="apache_beam.transforms.util.WithKeys">
<code class="descclassname">apache_beam.transforms.util.</code><code class="descname">WithKeys</code><span class="sig-paren">(</span><em>pcoll</em>, <em>k</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#WithKeys"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.WithKeys" title="Permalink to this definition"></a></dt>
<dd><p>PTransform that takes a PCollection, and either a constant key or a
callable, and returns a PCollection of (K, V), where each of the values in
the input PCollection has been paired with either the constant key or a key
computed from the value.</p>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.GroupIntoBatches">
<em class="property">class </em><code class="descclassname">apache_beam.transforms.util.</code><code class="descname">GroupIntoBatches</code><span class="sig-paren">(</span><em>batch_size</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#GroupIntoBatches"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.GroupIntoBatches" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>PTransform that batches the input into desired batch size. Elements are
buffered until they are equal to batch size provided in the argument at which
point they are output to the output Pcollection.</p>
<p>Windows are preserved (batches will contain elements from the same window)</p>
<p>GroupIntoBatches is experimental. Its use case will depend on the runner if
it has support of States and Timers.</p>
<p>Create a new GroupIntoBatches with batch size.</p>
<table class="docutils field-list" frame="void" rules="none">
<col class="field-name" />
<col class="field-body" />
<tbody valign="top">
<tr class="field-odd field"><th class="field-name">Parameters:</th><td class="field-body"><strong>batch_size</strong> – (required) How many elements should be in a batch</td>
</tr>
</tbody>
</table>
<dl class="method">
<dt id="apache_beam.transforms.util.GroupIntoBatches.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#GroupIntoBatches.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.GroupIntoBatches.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.ToString">
<em class="property">class </em><code class="descclassname">apache_beam.transforms.util.</code><code class="descname">ToString</code><a class="reference internal" href="_modules/apache_beam/transforms/util.html#ToString"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.ToString" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <code class="xref py py-class docutils literal"><span class="pre">future.types.newobject.newobject</span></code></p>
<p>PTransform for converting a PCollection element, KV or PCollection Iterable
to string.</p>
<dl class="class">
<dt id="apache_beam.transforms.util.ToString.Kvs">
<em class="property">class </em><code class="descname">Kvs</code><span class="sig-paren">(</span><em>delimiter=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#ToString.Kvs"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.ToString.Kvs" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>Transforms each element of the PCollection to a string on the key followed
by the specific delimiter and the value.</p>
<dl class="method">
<dt id="apache_beam.transforms.util.ToString.Kvs.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#ToString.Kvs.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.ToString.Kvs.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.ToString.Element">
<em class="property">class </em><code class="descname">Element</code><span class="sig-paren">(</span><em>delimiter=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#ToString.Element"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.ToString.Element" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>Transforms each element of the PCollection to a string.</p>
<dl class="method">
<dt id="apache_beam.transforms.util.ToString.Element.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#ToString.Element.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.ToString.Element.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.ToString.Iterables">
<em class="property">class </em><code class="descname">Iterables</code><span class="sig-paren">(</span><em>delimiter=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#ToString.Iterables"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.ToString.Iterables" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>Transforms each item in the iterable of the input of PCollection to a
string. There is no trailing delimiter.</p>
<dl class="method">
<dt id="apache_beam.transforms.util.ToString.Iterables.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#ToString.Iterables.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.ToString.Iterables.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.Reify">
<em class="property">class </em><code class="descclassname">apache_beam.transforms.util.</code><code class="descname">Reify</code><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <code class="xref py py-class docutils literal"><span class="pre">future.types.newobject.newobject</span></code></p>
<p>PTransforms for converting between explicit and implicit form of various
Beam values.</p>
<dl class="class">
<dt id="apache_beam.transforms.util.Reify.Timestamp">
<em class="property">class </em><code class="descname">Timestamp</code><span class="sig-paren">(</span><em>label=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.Timestamp"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.Timestamp" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>PTransform to wrap a value in a TimestampedValue with it’s
associated timestamp.</p>
<dl class="staticmethod">
<dt id="apache_beam.transforms.util.Reify.Timestamp.add_timestamp_info">
<em class="property">static </em><code class="descname">add_timestamp_info</code><span class="sig-paren">(</span><em>element</em>, <em>timestamp=TimestampParam</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.Timestamp.add_timestamp_info"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.Timestamp.add_timestamp_info" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="method">
<dt id="apache_beam.transforms.util.Reify.Timestamp.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.Timestamp.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.Timestamp.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.Reify.Window">
<em class="property">class </em><code class="descname">Window</code><span class="sig-paren">(</span><em>label=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.Window"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.Window" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>PTransform to convert an element in a PCollection into a tuple of
(element, timestamp, window), wrapped in a TimestampedValue with it’s
associated timestamp.</p>
<dl class="staticmethod">
<dt id="apache_beam.transforms.util.Reify.Window.add_window_info">
<em class="property">static </em><code class="descname">add_window_info</code><span class="sig-paren">(</span><em>element</em>, <em>timestamp=TimestampParam</em>, <em>window=WindowParam</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.Window.add_window_info"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.Window.add_window_info" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="method">
<dt id="apache_beam.transforms.util.Reify.Window.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.Window.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.Window.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.Reify.TimestampInValue">
<em class="property">class </em><code class="descname">TimestampInValue</code><span class="sig-paren">(</span><em>label=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.TimestampInValue"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.TimestampInValue" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>PTransform to wrap the Value in a KV pair in a TimestampedValue with
the element’s associated timestamp.</p>
<dl class="staticmethod">
<dt id="apache_beam.transforms.util.Reify.TimestampInValue.add_timestamp_info">
<em class="property">static </em><code class="descname">add_timestamp_info</code><span class="sig-paren">(</span><em>element</em>, <em>timestamp=TimestampParam</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.TimestampInValue.add_timestamp_info"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.TimestampInValue.add_timestamp_info" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="method">
<dt id="apache_beam.transforms.util.Reify.TimestampInValue.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.TimestampInValue.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.TimestampInValue.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
<dl class="class">
<dt id="apache_beam.transforms.util.Reify.WindowInValue">
<em class="property">class </em><code class="descname">WindowInValue</code><span class="sig-paren">(</span><em>label=None</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.WindowInValue"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.WindowInValue" title="Permalink to this definition"></a></dt>
<dd><p>Bases: <a class="reference internal" href="apache_beam.transforms.ptransform.html#apache_beam.transforms.ptransform.PTransform" title="apache_beam.transforms.ptransform.PTransform"><code class="xref py py-class docutils literal"><span class="pre">apache_beam.transforms.ptransform.PTransform</span></code></a></p>
<p>PTransform to convert the Value in a KV pair into a tuple of
(value, timestamp, window), with the whole element being wrapped inside a
TimestampedValue.</p>
<dl class="staticmethod">
<dt id="apache_beam.transforms.util.Reify.WindowInValue.add_window_info">
<em class="property">static </em><code class="descname">add_window_info</code><span class="sig-paren">(</span><em>element</em>, <em>timestamp=TimestampParam</em>, <em>window=WindowParam</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.WindowInValue.add_window_info"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.WindowInValue.add_window_info" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
<dl class="method">
<dt id="apache_beam.transforms.util.Reify.WindowInValue.expand">
<code class="descname">expand</code><span class="sig-paren">(</span><em>pcoll</em><span class="sig-paren">)</span><a class="reference internal" href="_modules/apache_beam/transforms/util.html#Reify.WindowInValue.expand"><span class="viewcode-link">[source]</span></a><a class="headerlink" href="#apache_beam.transforms.util.Reify.WindowInValue.expand" title="Permalink to this definition"></a></dt>
<dd></dd></dl>
</dd></dl>
</dd></dl>
</div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<div class="rst-footer-buttons" role="navigation" aria-label="footer navigation">
<a href="apache_beam.transforms.window.html" class="btn btn-neutral float-right" title="apache_beam.transforms.window module" accesskey="n" rel="next">Next <span class="fa fa-arrow-circle-right"></span></a>
<a href="apache_beam.transforms.userstate.html" class="btn btn-neutral" title="apache_beam.transforms.userstate module" accesskey="p" rel="prev"><span class="fa fa-arrow-circle-left"></span> Previous</a>
</div>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright .
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
var DOCUMENTATION_OPTIONS = {
URL_ROOT:'./',
VERSION:'',
COLLAPSE_INDEX:false,
FILE_SUFFIX:'.html',
HAS_SOURCE: true,
SOURCELINK_SUFFIX: '.txt'
};
</script>
<script type="text/javascript" src="_static/jquery.js"></script>
<script type="text/javascript" src="_static/underscore.js"></script>
<script type="text/javascript" src="_static/doctools.js"></script>
<script type="text/javascript" src="_static/js/theme.js"></script>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.StickyNav.enable();
});
</script>
</body>
</html>