blob: e88dbd449aabee95e93283e867423ac08b6ded27 [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 &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="Module code" href="../../index.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>
<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"><a class="reference internal" href="../../../apache_beam.transforms.html">apache_beam.transforms package</a></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="../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.transforms.util</li>
<li class="wy-breadcrumbs-aside">
</li>
</ul>
<hr/>
</div>
<div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article">
<div itemprop="articleBody">
<h1>Source code for apache_beam.transforms.util</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="sd">&quot;&quot;&quot;Simple utility PTransforms.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">division</span>
<span class="kn">import</span> <span class="nn">collections</span>
<span class="kn">import</span> <span class="nn">contextlib</span>
<span class="kn">import</span> <span class="nn">random</span>
<span class="kn">import</span> <span class="nn">time</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">object</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">range</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">zip</span>
<span class="kn">from</span> <span class="nn">future.utils</span> <span class="k">import</span> <span class="n">itervalues</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">typehints</span>
<span class="kn">from</span> <span class="nn">apache_beam.metrics</span> <span class="k">import</span> <span class="n">Metrics</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="k">import</span> <span class="n">window</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">CombinePerKey</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">DoFn</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">FlatMap</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">Flatten</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">GroupByKey</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">Map</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">ParDo</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">Windowing</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="k">import</span> <span class="n">PTransform</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="k">import</span> <span class="n">ptransform_fn</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="k">import</span> <span class="n">AccumulationMode</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="k">import</span> <span class="n">AfterCount</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="k">import</span> <span class="n">NonMergingWindowFn</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="k">import</span> <span class="n">TimestampCombiner</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="k">import</span> <span class="n">TimestampedValue</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="k">import</span> <span class="n">windowed_value</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span>
<span class="s1">&#39;BatchElements&#39;</span><span class="p">,</span>
<span class="s1">&#39;CoGroupByKey&#39;</span><span class="p">,</span>
<span class="s1">&#39;Keys&#39;</span><span class="p">,</span>
<span class="s1">&#39;KvSwap&#39;</span><span class="p">,</span>
<span class="s1">&#39;RemoveDuplicates&#39;</span><span class="p">,</span>
<span class="s1">&#39;Reshuffle&#39;</span><span class="p">,</span>
<span class="s1">&#39;Values&#39;</span><span class="p">,</span>
<span class="p">]</span>
<span class="n">K</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">TypeVariable</span><span class="p">(</span><span class="s1">&#39;K&#39;</span><span class="p">)</span>
<span class="n">V</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">TypeVariable</span><span class="p">(</span><span class="s1">&#39;V&#39;</span><span class="p">)</span>
<span class="n">T</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">TypeVariable</span><span class="p">(</span><span class="s1">&#39;T&#39;</span><span class="p">)</span>
<div class="viewcode-block" id="CoGroupByKey"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.CoGroupByKey">[docs]</a><span class="k">class</span> <span class="nc">CoGroupByKey</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Groups results across several PCollections by key.</span>
<span class="sd"> Given an input dict of serializable keys (called &quot;tags&quot;) to 0 or more</span>
<span class="sd"> PCollections of (key, value) tuples, it creates a single output PCollection</span>
<span class="sd"> of (key, value) tuples whose keys are the unique input keys from all inputs,</span>
<span class="sd"> and whose values are dicts mapping each tag to an iterable of whatever values</span>
<span class="sd"> were under the key in the corresponding PCollection, in this manner::</span>
<span class="sd"> (&#39;some key&#39;, {&#39;tag1&#39;: [&#39;value 1 under &quot;some key&quot; in pcoll1&#39;,</span>
<span class="sd"> &#39;value 2 under &quot;some key&quot; in pcoll1&#39;,</span>
<span class="sd"> ...],</span>
<span class="sd"> &#39;tag2&#39;: ... ,</span>
<span class="sd"> ... })</span>
<span class="sd"> For example, given:</span>
<span class="sd"> {&#39;tag1&#39;: pc1, &#39;tag2&#39;: pc2, 333: pc3}</span>
<span class="sd"> where:</span>
<span class="sd"> pc1 = [(k1, v1)]</span>
<span class="sd"> pc2 = []</span>
<span class="sd"> pc3 = [(k1, v31), (k1, v32), (k2, v33)]</span>
<span class="sd"> The output PCollection would be:</span>
<span class="sd"> [(k1, {&#39;tag1&#39;: [v1], &#39;tag2&#39;: [], 333: [v31, v32]}),</span>
<span class="sd"> (k2, {&#39;tag1&#39;: [], &#39;tag2&#39;: [], 333: [v33]})]</span>
<span class="sd"> CoGroupByKey also works for tuples, lists, or other flat iterables of</span>
<span class="sd"> PCollections, in which case the values of the resulting PCollections</span>
<span class="sd"> will be tuples whose nth value is the list of values from the nth</span>
<span class="sd"> PCollection---conceptually, the &quot;tags&quot; are the indices into the input.</span>
<span class="sd"> Thus, for this input::</span>
<span class="sd"> (pc1, pc2, pc3)</span>
<span class="sd"> the output would be::</span>
<span class="sd"> [(k1, ([v1], [], [v31, v32]),</span>
<span class="sd"> (k2, ([], [], [v33]))]</span>
<span class="sd"> Attributes:</span>
<span class="sd"> **kwargs: Accepts a single named argument &quot;pipeline&quot;, which specifies the</span>
<span class="sd"> pipeline that &quot;owns&quot; this PTransform. Ordinarily CoGroupByKey can obtain</span>
<span class="sd"> this information from one of the input PCollections, but if there are none</span>
<span class="sd"> (or if there&#39;s a chance there may be none), this argument is the only way</span>
<span class="sd"> to provide pipeline information, and should be considered mandatory.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">CoGroupByKey</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="s1">&#39;pipeline&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">if</span> <span class="n">kwargs</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Unexpected keyword arguments: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="nb">list</span><span class="p">(</span><span class="n">kwargs</span><span class="o">.</span><span class="n">keys</span><span class="p">()))</span>
<span class="k">def</span> <span class="nf">_extract_input_pvalues</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalueish</span><span class="p">):</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># If this works, it&#39;s a dict.</span>
<span class="k">return</span> <span class="n">pvalueish</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">itervalues</span><span class="p">(</span><span class="n">pvalueish</span><span class="p">))</span>
<span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span>
<span class="n">pcolls</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">pvalueish</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pcolls</span><span class="p">,</span> <span class="n">pcolls</span>
<div class="viewcode-block" id="CoGroupByKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.CoGroupByKey.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcolls</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Performs CoGroupByKey on argument pcolls; see class docstring.&quot;&quot;&quot;</span>
<span class="c1"># For associating values in K-V pairs with the PCollections they came from.</span>
<span class="k">def</span> <span class="nf">_pair_tag_with_value</span><span class="p">(</span><span class="n">key_value</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">)</span> <span class="o">=</span> <span class="n">key_value</span>
<span class="k">return</span> <span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">))</span>
<span class="c1"># Creates the key, value pairs for the output PCollection. Values are either</span>
<span class="c1"># lists or dicts (per the class docstring), initialized by the result of</span>
<span class="c1"># result_ctor(result_ctor_arg).</span>
<span class="k">def</span> <span class="nf">_merge_tagged_vals_under_key</span><span class="p">(</span><span class="n">key_grouped</span><span class="p">,</span> <span class="n">result_ctor</span><span class="p">,</span>
<span class="n">result_ctor_arg</span><span class="p">):</span>
<span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">grouped</span><span class="p">)</span> <span class="o">=</span> <span class="n">key_grouped</span>
<span class="n">result_value</span> <span class="o">=</span> <span class="n">result_ctor</span><span class="p">(</span><span class="n">result_ctor_arg</span><span class="p">)</span>
<span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">grouped</span><span class="p">:</span>
<span class="n">result_value</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">return</span> <span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">result_value</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># If pcolls is a dict, we turn it into (tag, pcoll) pairs for use in the</span>
<span class="c1"># general-purpose code below. The result value constructor creates dicts</span>
<span class="c1"># whose keys are the tags.</span>
<span class="n">result_ctor_arg</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">pcolls</span><span class="p">)</span>
<span class="n">result_ctor</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">tags</span><span class="p">:</span> <span class="nb">dict</span><span class="p">((</span><span class="n">tag</span><span class="p">,</span> <span class="p">[])</span> <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="n">tags</span><span class="p">)</span>
<span class="n">pcolls</span> <span class="o">=</span> <span class="n">pcolls</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span>
<span class="c1"># Otherwise, pcolls is a list/tuple, so we turn it into (index, pcoll)</span>
<span class="c1"># pairs. The result value constructor makes tuples with len(pcolls) slots.</span>
<span class="n">pcolls</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">enumerate</span><span class="p">(</span><span class="n">pcolls</span><span class="p">))</span>
<span class="n">result_ctor_arg</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">pcolls</span><span class="p">)</span>
<span class="n">result_ctor</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">size</span><span class="p">:</span> <span class="nb">tuple</span><span class="p">([]</span> <span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">size</span><span class="p">))</span>
<span class="c1"># Check input PCollections for PCollection-ness, and that they all belong</span>
<span class="c1"># to the same pipeline.</span>
<span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_check_pcollection</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">:</span>
<span class="k">assert</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span>
<span class="k">return</span> <span class="p">([</span><span class="n">pcoll</span> <span class="o">|</span> <span class="s1">&#39;pair_with_</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">tag</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="n">_pair_tag_with_value</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="p">]</span>
<span class="o">|</span> <span class="n">Flatten</span><span class="p">(</span><span class="n">pipeline</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span>
<span class="o">|</span> <span class="n">GroupByKey</span><span class="p">()</span>
<span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="n">_merge_tagged_vals_under_key</span><span class="p">,</span> <span class="n">result_ctor</span><span class="p">,</span> <span class="n">result_ctor_arg</span><span class="p">))</span></div></div>
<div class="viewcode-block" id="Keys"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Keys">[docs]</a><span class="k">def</span> <span class="nf">Keys</span><span class="p">(</span><span class="n">label</span><span class="o">=</span><span class="s1">&#39;Keys&#39;</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Produces a PCollection of first elements of 2-tuples in a PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">k_v</span><span class="p">:</span> <span class="n">k_v</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span></div>
<div class="viewcode-block" id="Values"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Values">[docs]</a><span class="k">def</span> <span class="nf">Values</span><span class="p">(</span><span class="n">label</span><span class="o">=</span><span class="s1">&#39;Values&#39;</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Produces a PCollection of second elements of 2-tuples in a PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">k_v1</span><span class="p">:</span> <span class="n">k_v1</span><span class="p">[</span><span class="mi">1</span><span class="p">])</span></div>
<div class="viewcode-block" id="KvSwap"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.KvSwap">[docs]</a><span class="k">def</span> <span class="nf">KvSwap</span><span class="p">(</span><span class="n">label</span><span class="o">=</span><span class="s1">&#39;KvSwap&#39;</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Produces a PCollection reversing 2-tuples in a PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">k_v2</span><span class="p">:</span> <span class="p">(</span><span class="n">k_v2</span><span class="p">[</span><span class="mi">1</span><span class="p">],</span> <span class="n">k_v2</span><span class="p">[</span><span class="mi">0</span><span class="p">]))</span></div>
<span class="nd">@ptransform_fn</span>
<span class="k">def</span> <span class="nf">RemoveDuplicates</span><span class="p">(</span><span class="n">pcoll</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Produces a PCollection containing the unique elements of a PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="p">(</span><span class="n">pcoll</span>
<span class="o">|</span> <span class="s1">&#39;ToPairs&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span>
<span class="o">|</span> <span class="s1">&#39;Group&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">CombinePerKey</span><span class="p">(</span><span class="k">lambda</span> <span class="n">vs</span><span class="p">:</span> <span class="kc">None</span><span class="p">)</span>
<span class="o">|</span> <span class="s1">&#39;RemoveDuplicates&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Keys</span><span class="p">())</span>
<span class="k">class</span> <span class="nc">_BatchSizeEstimator</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Estimates the best size for batches given historical timing.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">_MAX_DATA_POINTS</span> <span class="o">=</span> <span class="mi">100</span>
<span class="n">_MAX_GROWTH_FACTOR</span> <span class="o">=</span> <span class="mi">2</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span>
<span class="n">min_batch_size</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span>
<span class="n">max_batch_size</span><span class="o">=</span><span class="mi">1000</span><span class="p">,</span>
<span class="n">target_batch_overhead</span><span class="o">=.</span><span class="mi">1</span><span class="p">,</span>
<span class="n">target_batch_duration_secs</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span>
<span class="n">variance</span><span class="o">=</span><span class="mf">0.25</span><span class="p">,</span>
<span class="n">clock</span><span class="o">=</span><span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">):</span>
<span class="k">if</span> <span class="n">min_batch_size</span> <span class="o">&gt;</span> <span class="n">max_batch_size</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Minimum (</span><span class="si">%s</span><span class="s2">) must not be greater than maximum (</span><span class="si">%s</span><span class="s2">)&quot;</span> <span class="o">%</span> <span class="p">(</span>
<span class="n">min_batch_size</span><span class="p">,</span> <span class="n">max_batch_size</span><span class="p">))</span>
<span class="k">if</span> <span class="n">target_batch_overhead</span> <span class="ow">and</span> <span class="ow">not</span> <span class="mi">0</span> <span class="o">&lt;</span> <span class="n">target_batch_overhead</span> <span class="o">&lt;=</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;target_batch_overhead (</span><span class="si">%s</span><span class="s2">) must be between 0 and 1&quot;</span> <span class="o">%</span> <span class="p">(</span>
<span class="n">target_batch_overhead</span><span class="p">))</span>
<span class="k">if</span> <span class="n">target_batch_duration_secs</span> <span class="ow">and</span> <span class="n">target_batch_duration_secs</span> <span class="o">&lt;=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;target_batch_duration_secs (</span><span class="si">%s</span><span class="s2">) must be positive&quot;</span> <span class="o">%</span> <span class="p">(</span>
<span class="n">target_batch_duration_secs</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">max</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">target_batch_overhead</span><span class="p">,</span> <span class="n">target_batch_duration_secs</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;At least one of target_batch_overhead or &quot;</span>
<span class="s2">&quot;target_batch_duration_secs must be positive.&quot;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">=</span> <span class="n">min_batch_size</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span> <span class="o">=</span> <span class="n">max_batch_size</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_overhead</span> <span class="o">=</span> <span class="n">target_batch_overhead</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs</span> <span class="o">=</span> <span class="n">target_batch_duration_secs</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_variance</span> <span class="o">=</span> <span class="n">variance</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_clock</span> <span class="o">=</span> <span class="n">clock</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_data</span> <span class="o">=</span> <span class="p">[]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_ignore_next_timing</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_size_distribution</span> <span class="o">=</span> <span class="n">Metrics</span><span class="o">.</span><span class="n">distribution</span><span class="p">(</span>
<span class="s1">&#39;BatchElements&#39;</span><span class="p">,</span> <span class="s1">&#39;batch_size&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_time_distribution</span> <span class="o">=</span> <span class="n">Metrics</span><span class="o">.</span><span class="n">distribution</span><span class="p">(</span>
<span class="s1">&#39;BatchElements&#39;</span><span class="p">,</span> <span class="s1">&#39;msec_per_batch&#39;</span><span class="p">)</span>
<span class="c1"># Beam distributions only accept integer values, so we use this to</span>
<span class="c1"># accumulate under-reported values until they add up to whole milliseconds.</span>
<span class="c1"># (Milliseconds are chosen because that&#39;s conventionally used elsewhere in</span>
<span class="c1"># profiling-style counters.)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_remainder_msecs</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">def</span> <span class="nf">ignore_next_timing</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Call to indicate the next timing should be ignored.</span>
<span class="sd"> For example, the first emit of a ParDo operation is known to be anomalous</span>
<span class="sd"> due to setup that may occur.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_ignore_next_timing</span> <span class="o">=</span> <span class="kc">False</span>
<span class="nd">@contextlib</span><span class="o">.</span><span class="n">contextmanager</span>
<span class="k">def</span> <span class="nf">record_time</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">batch_size</span><span class="p">):</span>
<span class="n">start</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_clock</span><span class="p">()</span>
<span class="k">yield</span>
<span class="n">elapsed</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_clock</span><span class="p">()</span> <span class="o">-</span> <span class="n">start</span>
<span class="n">elapsed_msec</span> <span class="o">=</span> <span class="mf">1e3</span> <span class="o">*</span> <span class="n">elapsed</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_remainder_msecs</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_size_distribution</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">batch_size</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_time_distribution</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="nb">int</span><span class="p">(</span><span class="n">elapsed_msec</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_remainder_msecs</span> <span class="o">=</span> <span class="n">elapsed_msec</span> <span class="o">-</span> <span class="nb">int</span><span class="p">(</span><span class="n">elapsed_msec</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ignore_next_timing</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_ignore_next_timing</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">batch_size</span><span class="p">,</span> <span class="n">elapsed</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_MAX_DATA_POINTS</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_thin_data</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_thin_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># Make sure we don&#39;t change the parity of len(self._data)</span>
<span class="c1"># As it&#39;s used below to alternate jitter.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">randrange</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)</span> <span class="o">//</span> <span class="mi">4</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">randrange</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)</span> <span class="o">//</span> <span class="mi">2</span><span class="p">))</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">linear_regression_no_numpy</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">):</span>
<span class="c1"># Least squares fit for y = a + bx over all points.</span>
<span class="n">n</span> <span class="o">=</span> <span class="nb">float</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">xs</span><span class="p">))</span>
<span class="n">xbar</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">xs</span><span class="p">)</span> <span class="o">/</span> <span class="n">n</span>
<span class="n">ybar</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">ys</span><span class="p">)</span> <span class="o">/</span> <span class="n">n</span>
<span class="n">b</span> <span class="o">=</span> <span class="p">(</span><span class="nb">sum</span><span class="p">([(</span><span class="n">x</span> <span class="o">-</span> <span class="n">xbar</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">y</span> <span class="o">-</span> <span class="n">ybar</span><span class="p">)</span> <span class="k">for</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">)])</span>
<span class="o">/</span> <span class="nb">sum</span><span class="p">([(</span><span class="n">x</span> <span class="o">-</span> <span class="n">xbar</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">xs</span><span class="p">]))</span>
<span class="n">a</span> <span class="o">=</span> <span class="n">ybar</span> <span class="o">-</span> <span class="n">b</span> <span class="o">*</span> <span class="n">xbar</span>
<span class="k">return</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">linear_regression_numpy</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">):</span>
<span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span>
<span class="kn">import</span> <span class="nn">numpy</span> <span class="k">as</span> <span class="nn">np</span>
<span class="kn">from</span> <span class="nn">numpy</span> <span class="k">import</span> <span class="nb">sum</span>
<span class="n">xs</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">asarray</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="nb">float</span><span class="p">)</span>
<span class="n">ys</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">asarray</span><span class="p">(</span><span class="n">ys</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="nb">float</span><span class="p">)</span>
<span class="c1"># First do a simple least squares fit for y = a + bx over all points.</span>
<span class="n">b</span><span class="p">,</span> <span class="n">a</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">polyfit</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span>
<span class="n">n</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">xs</span><span class="p">)</span>
<span class="k">if</span> <span class="n">n</span> <span class="o">&lt;</span> <span class="mi">10</span><span class="p">:</span>
<span class="k">return</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Refine this by throwing out outliers, according to Cook&#39;s distance.</span>
<span class="c1"># https://en.wikipedia.org/wiki/Cook%27s_distance</span>
<span class="n">sum_x</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">xs</span><span class="p">)</span>
<span class="n">sum_x2</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">xs</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span>
<span class="n">errs</span> <span class="o">=</span> <span class="n">a</span> <span class="o">+</span> <span class="n">b</span> <span class="o">*</span> <span class="n">xs</span> <span class="o">-</span> <span class="n">ys</span>
<span class="n">s2</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">errs</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> <span class="o">/</span> <span class="p">(</span><span class="n">n</span> <span class="o">-</span> <span class="mi">2</span><span class="p">)</span>
<span class="k">if</span> <span class="n">s2</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="c1"># It&#39;s an exact fit!</span>
<span class="k">return</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span>
<span class="n">h</span> <span class="o">=</span> <span class="p">(</span><span class="n">sum_x2</span> <span class="o">-</span> <span class="mi">2</span> <span class="o">*</span> <span class="n">sum_x</span> <span class="o">*</span> <span class="n">xs</span> <span class="o">+</span> <span class="n">n</span> <span class="o">*</span> <span class="n">xs</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> <span class="o">/</span> <span class="p">(</span><span class="n">n</span> <span class="o">*</span> <span class="n">sum_x2</span> <span class="o">-</span> <span class="n">sum_x</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span>
<span class="n">cook_ds</span> <span class="o">=</span> <span class="mf">0.5</span> <span class="o">/</span> <span class="n">s2</span> <span class="o">*</span> <span class="n">errs</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="p">(</span><span class="n">h</span> <span class="o">/</span> <span class="p">(</span><span class="mi">1</span> <span class="o">-</span> <span class="n">h</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span>
<span class="c1"># Re-compute the regression, excluding those points with Cook&#39;s distance</span>
<span class="c1"># greater than 0.5, and weighting by the inverse of x to give a more</span>
<span class="c1"># stable y-intercept (as small batches have relatively more information</span>
<span class="c1"># about the fixed overhead).</span>
<span class="n">weight</span> <span class="o">=</span> <span class="p">(</span><span class="n">cook_ds</span> <span class="o">&lt;=</span> <span class="mf">0.5</span><span class="p">)</span> <span class="o">/</span> <span class="n">xs</span>
<span class="n">b</span><span class="p">,</span> <span class="n">a</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">polyfit</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">,</span> <span class="mi">1</span><span class="p">,</span> <span class="n">w</span><span class="o">=</span><span class="n">weight</span><span class="p">)</span>
<span class="k">return</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span>
<span class="kn">import</span> <span class="nn">numpy</span> <span class="k">as</span> <span class="nn">np</span>
<span class="n">linear_regression</span> <span class="o">=</span> <span class="n">linear_regression_numpy</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="n">linear_regression</span> <span class="o">=</span> <span class="n">linear_regression_no_numpy</span>
<span class="k">def</span> <span class="nf">next_batch_size</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span>
<span class="k">elif</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)</span> <span class="o">&lt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span>
<span class="k">elif</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)</span> <span class="o">&lt;</span> <span class="mi">2</span><span class="p">:</span>
<span class="c1"># Force some variety so we have distinct batch sizes on which to do</span>
<span class="c1"># linear regression below.</span>
<span class="k">return</span> <span class="nb">int</span><span class="p">(</span><span class="nb">max</span><span class="p">(</span>
<span class="nb">min</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">*</span> <span class="bp">self</span><span class="o">.</span><span class="n">_MAX_GROWTH_FACTOR</span><span class="p">),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">+</span> <span class="mi">1</span><span class="p">))</span>
<span class="c1"># There tends to be a lot of noise in the top quantile, which also</span>
<span class="c1"># has outsided influence in the regression. If we have enough data,</span>
<span class="c1"># Simply declare the top 20% to be outliers.</span>
<span class="n">trimmed_data</span> <span class="o">=</span> <span class="nb">sorted</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)[:</span><span class="nb">max</span><span class="p">(</span><span class="mi">20</span><span class="p">,</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)</span> <span class="o">*</span> <span class="mi">4</span> <span class="o">//</span> <span class="mi">5</span><span class="p">)]</span>
<span class="c1"># Linear regression for y = a + bx, where x is batch size and y is time.</span>
<span class="n">xs</span><span class="p">,</span> <span class="n">ys</span> <span class="o">=</span> <span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">trimmed_data</span><span class="p">)</span>
<span class="n">a</span><span class="p">,</span> <span class="n">b</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">linear_regression</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">)</span>
<span class="c1"># Avoid nonsensical or division-by-zero errors below due to noise.</span>
<span class="n">a</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="mf">1e-10</span><span class="p">)</span>
<span class="n">b</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="n">b</span><span class="p">,</span> <span class="mf">1e-20</span><span class="p">)</span>
<span class="n">last_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">][</span><span class="mi">0</span><span class="p">]</span>
<span class="n">cap</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">last_batch_size</span> <span class="o">*</span> <span class="bp">self</span><span class="o">.</span><span class="n">_MAX_GROWTH_FACTOR</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">)</span>
<span class="n">target</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs</span><span class="p">:</span>
<span class="c1"># Solution to a + b*x = self._target_batch_duration_secs.</span>
<span class="n">target</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">target</span><span class="p">,</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs</span> <span class="o">-</span> <span class="n">a</span><span class="p">)</span> <span class="o">/</span> <span class="n">b</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_overhead</span><span class="p">:</span>
<span class="c1"># Solution to a / (a + b*x) = self._target_batch_overhead.</span>
<span class="n">target</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">target</span><span class="p">,</span> <span class="p">(</span><span class="n">a</span> <span class="o">/</span> <span class="n">b</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="mi">1</span> <span class="o">/</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_overhead</span> <span class="o">-</span> <span class="mi">1</span><span class="p">))</span>
<span class="c1"># Avoid getting stuck at a single batch size (especially the minimal</span>
<span class="c1"># batch size) which would not allow us to extrapolate to other batch</span>
<span class="c1"># sizes.</span>
<span class="c1"># Jitter alternates between 0 and 1.</span>
<span class="n">jitter</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)</span> <span class="o">%</span> <span class="mi">2</span>
<span class="c1"># Smear our samples across a range centered at the target.</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">10</span><span class="p">:</span>
<span class="n">target</span> <span class="o">+=</span> <span class="nb">int</span><span class="p">(</span><span class="n">target</span> <span class="o">*</span> <span class="bp">self</span><span class="o">.</span><span class="n">_variance</span> <span class="o">*</span> <span class="mi">2</span> <span class="o">*</span> <span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">random</span><span class="p">()</span> <span class="o">-</span> <span class="o">.</span><span class="mi">5</span><span class="p">))</span>
<span class="k">return</span> <span class="nb">int</span><span class="p">(</span><span class="nb">max</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">+</span> <span class="n">jitter</span><span class="p">,</span> <span class="nb">min</span><span class="p">(</span><span class="n">target</span><span class="p">,</span> <span class="n">cap</span><span class="p">)))</span>
<span class="k">class</span> <span class="nc">_GlobalWindowsBatchingDoFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">batch_size_estimator</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span> <span class="o">=</span> <span class="n">batch_size_estimator</span>
<span class="k">def</span> <span class="nf">start_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch</span> <span class="o">=</span> <span class="p">[]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span>
<span class="c1"># The first emit often involves non-trivial setup.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">ignore_next_timing</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">element</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch</span><span class="p">)</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span><span class="p">:</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span><span class="p">):</span>
<span class="k">yield</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch</span> <span class="o">=</span> <span class="p">[]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch</span><span class="p">:</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="o">.</span><span class="n">windowed_value</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span>
<span class="k">class</span> <span class="nc">_WindowAwareBatchingDoFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="n">_MAX_LIVE_WINDOWS</span> <span class="o">=</span> <span class="mi">10</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">batch_size_estimator</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span> <span class="o">=</span> <span class="n">batch_size_estimator</span>
<span class="k">def</span> <span class="nf">start_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batches</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span>
<span class="c1"># The first emit often involves non-trivial setup.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">ignore_next_timing</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">element</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">])</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span><span class="p">:</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">],</span> <span class="n">window</span><span class="o">.</span><span class="n">max_timestamp</span><span class="p">(),</span> <span class="p">(</span><span class="n">window</span><span class="p">,))</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span>
<span class="k">elif</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">)</span> <span class="o">&gt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_MAX_LIVE_WINDOWS</span><span class="p">:</span>
<span class="n">window</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="nb">sorted</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="o">.</span><span class="n">items</span><span class="p">(),</span>
<span class="n">key</span><span class="o">=</span><span class="k">lambda</span> <span class="n">window_batch</span><span class="p">:</span> <span class="nb">len</span><span class="p">(</span><span class="n">window_batch</span><span class="p">[</span><span class="mi">1</span><span class="p">]),</span>
<span class="n">reverse</span><span class="o">=</span><span class="kc">True</span><span class="p">)[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">],</span> <span class="n">window</span><span class="o">.</span><span class="n">max_timestamp</span><span class="p">(),</span> <span class="p">(</span><span class="n">window</span><span class="p">,))</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">for</span> <span class="n">window</span><span class="p">,</span> <span class="n">batch</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">if</span> <span class="n">batch</span><span class="p">:</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span>
<span class="n">batch</span><span class="p">,</span> <span class="n">window</span><span class="o">.</span><span class="n">max_timestamp</span><span class="p">(),</span> <span class="p">(</span><span class="n">window</span><span class="p">,))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batches</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span>
<div class="viewcode-block" id="BatchElements"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.BatchElements">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">List</span><span class="p">[</span><span class="n">T</span><span class="p">])</span>
<span class="k">class</span> <span class="nc">BatchElements</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A Transform that batches elements for amortized processing.</span>
<span class="sd"> This transform is designed to precede operations whose processing cost</span>
<span class="sd"> is of the form</span>
<span class="sd"> time = fixed_cost + num_elements * per_element_cost</span>
<span class="sd"> where the per element cost is (often significantly) smaller than the fixed</span>
<span class="sd"> cost and could be amortized over multiple elements. It consumes a PCollection</span>
<span class="sd"> of element type T and produces a PCollection of element type List[T].</span>
<span class="sd"> This transform attempts to find the best batch size between the minimim</span>
<span class="sd"> and maximum parameters by profiling the time taken by (fused) downstream</span>
<span class="sd"> operations. For a fixed batch size, set the min and max to be equal.</span>
<span class="sd"> Elements are batched per-window and batches emitted in the window</span>
<span class="sd"> corresponding to its contents.</span>
<span class="sd"> Args:</span>
<span class="sd"> min_batch_size: (optional) the smallest number of elements per batch</span>
<span class="sd"> max_batch_size: (optional) the largest number of elements per batch</span>
<span class="sd"> target_batch_overhead: (optional) a target for fixed_cost / time,</span>
<span class="sd"> as used in the formula above</span>
<span class="sd"> target_batch_duration_secs: (optional) a target for total time per bundle,</span>
<span class="sd"> in seconds</span>
<span class="sd"> variance: (optional) the permitted (relative) amount of deviation from the</span>
<span class="sd"> (estimated) ideal batch size used to produce a wider base for</span>
<span class="sd"> linear interpolation</span>
<span class="sd"> clock: (optional) an alternative to time.time for measuring the cost of</span>
<span class="sd"> donwstream operations (mostly for testing)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span>
<span class="n">min_batch_size</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span>
<span class="n">max_batch_size</span><span class="o">=</span><span class="mi">10000</span><span class="p">,</span>
<span class="n">target_batch_overhead</span><span class="o">=.</span><span class="mi">05</span><span class="p">,</span>
<span class="n">target_batch_duration_secs</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span>
<span class="n">variance</span><span class="o">=</span><span class="mf">0.25</span><span class="p">,</span>
<span class="n">clock</span><span class="o">=</span><span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span> <span class="o">=</span> <span class="n">_BatchSizeEstimator</span><span class="p">(</span>
<span class="n">min_batch_size</span><span class="o">=</span><span class="n">min_batch_size</span><span class="p">,</span>
<span class="n">max_batch_size</span><span class="o">=</span><span class="n">max_batch_size</span><span class="p">,</span>
<span class="n">target_batch_overhead</span><span class="o">=</span><span class="n">target_batch_overhead</span><span class="p">,</span>
<span class="n">target_batch_duration_secs</span><span class="o">=</span><span class="n">target_batch_duration_secs</span><span class="p">,</span>
<span class="n">variance</span><span class="o">=</span><span class="n">variance</span><span class="p">,</span>
<span class="n">clock</span><span class="o">=</span><span class="n">clock</span><span class="p">)</span>
<div class="viewcode-block" id="BatchElements.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.BatchElements.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">runner</span><span class="p">,</span> <span class="s1">&#39;is_streaming&#39;</span><span class="p">,</span> <span class="kc">False</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;Requires stateful processing (BEAM-2687)&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">windowing</span><span class="o">.</span><span class="n">is_default</span><span class="p">():</span>
<span class="c1"># This is the same logic as _GlobalWindowsBatchingDoFn, but optimized</span>
<span class="c1"># for that simpler case.</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">_GlobalWindowsBatchingDoFn</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">_WindowAwareBatchingDoFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="p">))</span></div></div>
<span class="k">class</span> <span class="nc">_IdentityWindowFn</span><span class="p">(</span><span class="n">NonMergingWindowFn</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Windowing function that preserves existing windows.</span>
<span class="sd"> To be used internally with the Reshuffle transform.</span>
<span class="sd"> Will raise an exception when used after DoFns that return TimestampedValue</span>
<span class="sd"> elements.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window_coder</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Create a new WindowFn with compatible coder.</span>
<span class="sd"> To be applied to PCollections with windows that are compatible with the</span>
<span class="sd"> given coder.</span>
<span class="sd"> Arguments:</span>
<span class="sd"> window_coder: coders.Coder object to be used on windows.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nb">super</span><span class="p">(</span><span class="n">_IdentityWindowFn</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="k">if</span> <span class="n">window_coder</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;window_coder should not be None&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window_coder</span> <span class="o">=</span> <span class="n">window_coder</span>
<span class="k">def</span> <span class="nf">assign</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">assign_context</span><span class="p">):</span>
<span class="k">if</span> <span class="n">assign_context</span><span class="o">.</span><span class="n">window</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;assign_context.window should not be None. &#39;</span>
<span class="s1">&#39;This might be due to a DoFn returning a TimestampedValue.&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="p">[</span><span class="n">assign_context</span><span class="o">.</span><span class="n">window</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">get_window_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_window_coder</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span>
<span class="k">class</span> <span class="nc">ReshufflePerKey</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;PTransform that returns a PCollection equivalent to its input,</span>
<span class="sd"> but operationally provides some of the side effects of a GroupByKey,</span>
<span class="sd"> in particular preventing fusion of the surrounding transforms,</span>
<span class="sd"> checkpointing, and deduplication by id.</span>
<span class="sd"> ReshufflePerKey is experimental. No backwards compatibility guarantees.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="n">windowing_saved</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">windowing</span>
<span class="k">if</span> <span class="n">windowing_saved</span><span class="o">.</span><span class="n">is_default</span><span class="p">():</span>
<span class="c1"># In this (common) case we can use a trivial trigger driver</span>
<span class="c1"># and avoid the (expensive) window param.</span>
<span class="n">globally_windowed</span> <span class="o">=</span> <span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="o">.</span><span class="n">windowed_value</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span>
<span class="n">window_fn</span> <span class="o">=</span> <span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="p">()</span>
<span class="n">MIN_TIMESTAMP</span> <span class="o">=</span> <span class="n">window</span><span class="o">.</span><span class="n">MIN_TIMESTAMP</span>
<span class="k">def</span> <span class="nf">reify_timestamps</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">):</span>
<span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">element</span>
<span class="k">if</span> <span class="n">timestamp</span> <span class="o">==</span> <span class="n">MIN_TIMESTAMP</span><span class="p">:</span>
<span class="n">timestamp</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">return</span> <span class="n">key</span><span class="p">,</span> <span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">restore_timestamps</span><span class="p">(</span><span class="n">element</span><span class="p">):</span>
<span class="n">key</span><span class="p">,</span> <span class="n">values</span> <span class="o">=</span> <span class="n">element</span>
<span class="k">return</span> <span class="p">[</span>
<span class="n">globally_windowed</span><span class="o">.</span><span class="n">with_value</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">))</span>
<span class="k">if</span> <span class="n">timestamp</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">else</span> <span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="o">.</span><span class="n">windowed_value</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">),</span> <span class="n">timestamp</span><span class="p">)</span>
<span class="k">for</span> <span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span> <span class="ow">in</span> <span class="n">values</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># The linter is confused.</span>
<span class="c1"># hash(1) is used to force &quot;runtime&quot; selection of _IdentityWindowFn</span>
<span class="c1"># pylint: disable=abstract-class-instantiated</span>
<span class="bp">cls</span> <span class="o">=</span> <span class="nb">hash</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span> <span class="ow">and</span> <span class="n">_IdentityWindowFn</span>
<span class="n">window_fn</span> <span class="o">=</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">windowing_saved</span><span class="o">.</span><span class="n">windowfn</span><span class="o">.</span><span class="n">get_window_coder</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">reify_timestamps</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">):</span>
<span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">element</span>
<span class="k">return</span> <span class="n">key</span><span class="p">,</span> <span class="n">TimestampedValue</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">restore_timestamps</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">):</span>
<span class="c1"># Pass the current window since _IdentityWindowFn wouldn&#39;t know how</span>
<span class="c1"># to generate it.</span>
<span class="n">key</span><span class="p">,</span> <span class="n">values</span> <span class="o">=</span> <span class="n">element</span>
<span class="k">return</span> <span class="p">[</span>
<span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span>
<span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="o">.</span><span class="n">value</span><span class="p">),</span> <span class="n">value</span><span class="o">.</span><span class="n">timestamp</span><span class="p">,</span> <span class="p">[</span><span class="n">window</span><span class="p">])</span>
<span class="k">for</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">values</span><span class="p">]</span>
<span class="n">ungrouped</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="n">reify_timestamps</span><span class="p">)</span>
<span class="n">ungrouped</span><span class="o">.</span><span class="n">_windowing</span> <span class="o">=</span> <span class="n">Windowing</span><span class="p">(</span>
<span class="n">window_fn</span><span class="p">,</span>
<span class="n">triggerfn</span><span class="o">=</span><span class="n">AfterCount</span><span class="p">(</span><span class="mi">1</span><span class="p">),</span>
<span class="n">accumulation_mode</span><span class="o">=</span><span class="n">AccumulationMode</span><span class="o">.</span><span class="n">DISCARDING</span><span class="p">,</span>
<span class="n">timestamp_combiner</span><span class="o">=</span><span class="n">TimestampCombiner</span><span class="o">.</span><span class="n">OUTPUT_AT_EARLIEST</span><span class="p">)</span>
<span class="n">result</span> <span class="o">=</span> <span class="p">(</span><span class="n">ungrouped</span>
<span class="o">|</span> <span class="n">GroupByKey</span><span class="p">()</span>
<span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">restore_timestamps</span><span class="p">))</span>
<span class="n">result</span><span class="o">.</span><span class="n">_windowing</span> <span class="o">=</span> <span class="n">windowing_saved</span>
<span class="k">return</span> <span class="n">result</span>
<div class="viewcode-block" id="Reshuffle"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reshuffle">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">Reshuffle</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;PTransform that returns a PCollection equivalent to its input,</span>
<span class="sd"> but operationally provides some of the side effects of a GroupByKey,</span>
<span class="sd"> in particular preventing fusion of the surrounding transforms,</span>
<span class="sd"> checkpointing, and deduplication by id.</span>
<span class="sd"> Reshuffle adds a temporary random key to each element, performs a</span>
<span class="sd"> ReshufflePerKey, and finally removes the temporary key.</span>
<span class="sd"> Reshuffle is experimental. No backwards compatibility guarantees.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="Reshuffle.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reshuffle.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span><span class="n">pcoll</span>
<span class="o">|</span> <span class="s1">&#39;AddRandomKeys&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">t</span><span class="p">:</span> <span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">getrandbits</span><span class="p">(</span><span class="mi">32</span><span class="p">),</span> <span class="n">t</span><span class="p">))</span>
<span class="o">|</span> <span class="n">ReshufflePerKey</span><span class="p">()</span>
<span class="o">|</span> <span class="s1">&#39;RemoveRandomKeys&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">t</span><span class="p">:</span> <span class="n">t</span><span class="p">[</span><span class="mi">1</span><span class="p">]))</span></div></div>
</pre></div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<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>