blob: b8d54ef000e1dc0338b2394e3834d079736ec4f4 [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.combiners &mdash; Apache Beam 2.47.0 documentation</title>
<script type="text/javascript" src="../../../_static/js/modernizr.min.js"></script>
<script type="text/javascript" id="documentation_options" data-url_root="../../../" src="../../../_static/documentation_options.js"></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/language_data.js"></script>
<script async="async" type="text/javascript" src="https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.5/latest.js?config=TeX-AMS-MML_HTMLorMML"></script>
<script type="text/javascript" src="../../../_static/js/theme.js"></script>
<link rel="stylesheet" href="../../../_static/css/theme.css" type="text/css" />
<link rel="stylesheet" href="../../../_static/pygments.css" type="text/css" />
<link rel="index" title="Index" href="../../../genindex.html" />
<link rel="search" title="Search" href="../../../search.html" />
</head>
<body class="wy-body-for-nav">
<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 class="version">
2.47.0
</div>
<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.dataframe.html">apache_beam.dataframe 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.ml.html">apache_beam.ml 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.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>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.yaml.html">apache_beam.yaml 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>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" 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.combiners</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.combiners</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;A library of basic combiner PTransform subclasses.&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span> <span class="nn">copy</span>
<span class="kn">import</span> <span class="nn">heapq</span>
<span class="kn">import</span> <span class="nn">itertools</span>
<span class="kn">import</span> <span class="nn">operator</span>
<span class="kn">import</span> <span class="nn">random</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Dict</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Iterable</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">List</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Set</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Tuple</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TypeVar</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Union</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">apache_beam</span> <span class="kn">import</span> <span class="n">typehints</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">core</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">cy_combiners</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">ptransform</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">window</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.display</span> <span class="kn">import</span> <span class="n">DisplayDataItem</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints</span> <span class="kn">import</span> <span class="n">with_input_types</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints</span> <span class="kn">import</span> <span class="n">with_output_types</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.timestamp</span> <span class="kn">import</span> <span class="n">Duration</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.timestamp</span> <span class="kn">import</span> <span class="n">Timestamp</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span>
<span class="s1">&#39;Count&#39;</span><span class="p">,</span>
<span class="s1">&#39;Mean&#39;</span><span class="p">,</span>
<span class="s1">&#39;Sample&#39;</span><span class="p">,</span>
<span class="s1">&#39;Top&#39;</span><span class="p">,</span>
<span class="s1">&#39;ToDict&#39;</span><span class="p">,</span>
<span class="s1">&#39;ToList&#39;</span><span class="p">,</span>
<span class="s1">&#39;ToSet&#39;</span><span class="p">,</span>
<span class="s1">&#39;Latest&#39;</span><span class="p">,</span>
<span class="s1">&#39;CountCombineFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;MeanCombineFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;SampleCombineFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;TopCombineFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;ToDictCombineFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;ToListCombineFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;ToSetCombineFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;LatestCombineFn&#39;</span><span class="p">,</span>
<span class="p">]</span>
<span class="c1"># Type variables</span>
<span class="n">T</span> <span class="o">=</span> <span class="n">TypeVar</span><span class="p">(</span><span class="s1">&#39;T&#39;</span><span class="p">)</span>
<span class="n">K</span> <span class="o">=</span> <span class="n">TypeVar</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">TypeVar</span><span class="p">(</span><span class="s1">&#39;V&#39;</span><span class="p">)</span>
<span class="n">TimestampType</span> <span class="o">=</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="nb">float</span><span class="p">,</span> <span class="n">Timestamp</span><span class="p">,</span> <span class="n">Duration</span><span class="p">]</span>
<span class="k">class</span> <span class="nc">CombinerWithoutDefaults</span><span class="p">(</span><span class="n">ptransform</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Super class to inherit without_defaults to built-in Combiners.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">has_defaults</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span>
<span class="nb">super</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">has_defaults</span> <span class="o">=</span> <span class="n">has_defaults</span>
<span class="k">def</span> <span class="nf">with_defaults</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">has_defaults</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span>
<span class="n">new</span> <span class="o">=</span> <span class="n">copy</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span>
<span class="n">new</span><span class="o">.</span><span class="n">has_defaults</span> <span class="o">=</span> <span class="n">has_defaults</span>
<span class="k">return</span> <span class="n">new</span>
<span class="k">def</span> <span class="nf">without_defaults</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">with_defaults</span><span class="p">(</span><span class="kc">False</span><span class="p">)</span>
<div class="viewcode-block" id="Mean"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Mean">[docs]</a><span class="k">class</span> <span class="nc">Mean</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Combiners for computing arithmetic means of elements.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Mean.Globally"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Mean.Globally">[docs]</a> <span class="k">class</span> <span class="nc">Globally</span><span class="p">(</span><span class="n">CombinerWithoutDefaults</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;combiners.Mean.Globally computes the arithmetic mean of the elements.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Mean.Globally.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Mean.Globally.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="bp">self</span><span class="o">.</span><span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">MeanCombineFn</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">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">MeanCombineFn</span><span class="p">())</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div></div>
<div class="viewcode-block" id="Mean.PerKey"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Mean.PerKey">[docs]</a> <span class="k">class</span> <span class="nc">PerKey</span><span class="p">(</span><span class="n">ptransform</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;combiners.Mean.PerKey finds the means of the values for each key.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Mean.PerKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Mean.PerKey.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="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombinePerKey</span><span class="p">(</span><span class="n">MeanCombineFn</span><span class="p">())</span></div></div></div>
<span class="c1"># TODO(laolu): This type signature is overly restrictive. This should be</span>
<span class="c1"># more general.</span>
<div class="viewcode-block" id="MeanCombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.MeanCombineFn">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Union</span><span class="p">[</span><span class="nb">float</span><span class="p">,</span> <span class="nb">int</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">int64</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">float64</span><span class="p">])</span>
<span class="nd">@with_output_types</span><span class="p">(</span><span class="nb">float</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">MeanCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;CombineFn for computing an arithmetic mean.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="MeanCombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.MeanCombineFn.create_accumulator">[docs]</a> <span class="k">def</span> <span class="nf">create_accumulator</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span></div>
<div class="viewcode-block" id="MeanCombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.MeanCombineFn.add_input">[docs]</a> <span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">sum_count</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="p">(</span><span class="n">sum_</span><span class="p">,</span> <span class="n">count</span><span class="p">)</span> <span class="o">=</span> <span class="n">sum_count</span>
<span class="k">return</span> <span class="n">sum_</span> <span class="o">+</span> <span class="n">element</span><span class="p">,</span> <span class="n">count</span> <span class="o">+</span> <span class="mi">1</span></div>
<div class="viewcode-block" id="MeanCombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.MeanCombineFn.merge_accumulators">[docs]</a> <span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">):</span>
<span class="n">sums</span><span class="p">,</span> <span class="n">counts</span> <span class="o">=</span> <span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">accumulators</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">sum</span><span class="p">(</span><span class="n">sums</span><span class="p">),</span> <span class="nb">sum</span><span class="p">(</span><span class="n">counts</span><span class="p">)</span></div>
<div class="viewcode-block" id="MeanCombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.MeanCombineFn.extract_output">[docs]</a> <span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">sum_count</span><span class="p">):</span>
<span class="p">(</span><span class="n">sum_</span><span class="p">,</span> <span class="n">count</span><span class="p">)</span> <span class="o">=</span> <span class="n">sum_count</span>
<span class="k">if</span> <span class="n">count</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">float</span><span class="p">(</span><span class="s1">&#39;NaN&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">sum_</span> <span class="o">/</span> <span class="nb">float</span><span class="p">(</span><span class="n">count</span><span class="p">)</span></div>
<div class="viewcode-block" id="MeanCombineFn.for_input_type"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.MeanCombineFn.for_input_type">[docs]</a> <span class="k">def</span> <span class="nf">for_input_type</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">input_type</span><span class="p">):</span>
<span class="k">if</span> <span class="n">input_type</span> <span class="ow">is</span> <span class="nb">int</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">MeanInt64Fn</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">input_type</span> <span class="ow">is</span> <span class="nb">float</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">MeanFloatFn</span><span class="p">()</span>
<span class="k">return</span> <span class="bp">self</span></div></div>
<div class="viewcode-block" id="Count"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Count">[docs]</a><span class="k">class</span> <span class="nc">Count</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Combiners for counting elements.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Count.Globally"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Count.Globally">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</span><span class="nb">int</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">Globally</span><span class="p">(</span><span class="n">CombinerWithoutDefaults</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;combiners.Count.Globally counts the total number of elements.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Count.Globally.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Count.Globally.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="bp">self</span><span class="o">.</span><span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">CountCombineFn</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">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">CountCombineFn</span><span class="p">())</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div></div>
<div class="viewcode-block" id="Count.PerKey"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Count.PerKey">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Tuple</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">@with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="nb">int</span><span class="p">])</span>
<span class="k">class</span> <span class="nc">PerKey</span><span class="p">(</span><span class="n">ptransform</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;combiners.Count.PerKey counts how many elements each unique key has.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Count.PerKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Count.PerKey.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="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombinePerKey</span><span class="p">(</span><span class="n">CountCombineFn</span><span class="p">())</span></div></div>
<div class="viewcode-block" id="Count.PerElement"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Count.PerElement">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">T</span><span class="p">,</span> <span class="nb">int</span><span class="p">])</span>
<span class="k">class</span> <span class="nc">PerElement</span><span class="p">(</span><span class="n">ptransform</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;combiners.Count.PerElement counts how many times each element occurs.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Count.PerElement.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Count.PerElement.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="n">paired_with_void_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span><span class="p">,</span> <span class="n">Any</span><span class="p">]</span>
<span class="n">output_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span><span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span><span class="p">,</span> <span class="nb">int</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="p">(</span>
<span class="s1">&#39;</span><span class="si">%s</span><span class="s1">:PairWithVoid&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">core</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">paired_with_void_type</span><span class="p">))</span>
<span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombinePerKey</span><span class="p">(</span><span class="n">CountCombineFn</span><span class="p">())</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">output_type</span><span class="p">))</span></div></div></div>
<div class="viewcode-block" id="CountCombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.CountCombineFn">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Any</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</span><span class="nb">int</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">CountCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;CombineFn for computing PCollection size.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="CountCombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.CountCombineFn.create_accumulator">[docs]</a> <span class="k">def</span> <span class="nf">create_accumulator</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="mi">0</span></div>
<div class="viewcode-block" id="CountCombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.CountCombineFn.add_input">[docs]</a> <span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="k">return</span> <span class="n">accumulator</span> <span class="o">+</span> <span class="mi">1</span></div>
<div class="viewcode-block" id="CountCombineFn.add_inputs"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.CountCombineFn.add_inputs">[docs]</a> <span class="k">def</span> <span class="nf">add_inputs</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">elements</span><span class="p">):</span>
<span class="k">return</span> <span class="n">accumulator</span> <span class="o">+</span> <span class="nb">len</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="n">elements</span><span class="p">))</span></div>
<div class="viewcode-block" id="CountCombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.CountCombineFn.merge_accumulators">[docs]</a> <span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">sum</span><span class="p">(</span><span class="n">accumulators</span><span class="p">)</span></div>
<div class="viewcode-block" id="CountCombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.CountCombineFn.extract_output">[docs]</a> <span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">accumulator</span></div></div>
<div class="viewcode-block" id="Top"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top">[docs]</a><span class="k">class</span> <span class="nc">Top</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Combiners for obtaining extremal elements.&quot;&quot;&quot;</span>
<span class="c1"># pylint: disable=no-self-argument</span>
<div class="viewcode-block" id="Top.Of"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.Of">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</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">Of</span><span class="p">(</span><span class="n">CombinerWithoutDefaults</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Obtain a list of the compare-most N elements in a PCollection.</span>
<span class="sd"> This transform will retrieve the n greatest elements in the PCollection</span>
<span class="sd"> to which it is applied, where &quot;greatest&quot; is determined by the comparator</span>
<span class="sd"> function supplied as the compare argument.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Creates a global Top operation.</span>
<span class="sd"> The arguments &#39;key&#39; and &#39;reverse&#39; may be passed as keyword arguments,</span>
<span class="sd"> and have the same meaning as for Python&#39;s sort functions.</span>
<span class="sd"> Args:</span>
<span class="sd"> n: number of elements to extract from pcoll.</span>
<span class="sd"> key: (optional) a mapping of elements to a comparable key, similar to</span>
<span class="sd"> the key argument of Python&#39;s sorting methods.</span>
<span class="sd"> reverse: (optional) whether to order things smallest to largest, rather</span>
<span class="sd"> than largest to smallest</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nb">super</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">_n</span> <span class="o">=</span> <span class="n">n</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_key</span> <span class="o">=</span> <span class="n">key</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_reverse</span> <span class="o">=</span> <span class="n">reverse</span>
<div class="viewcode-block" id="Top.Of.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.Of.default_label">[docs]</a> <span class="k">def</span> <span class="nf">default_label</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;Top(</span><span class="si">%d</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span></div>
<div class="viewcode-block" id="Top.Of.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.Of.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="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 a more efficient global algorithm.</span>
<span class="n">top_per_bundle</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span>
<span class="n">_TopPerBundle</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reverse</span><span class="p">))</span>
<span class="c1"># If pcoll is empty, we can&#39;t guarantee that top_per_bundle</span>
<span class="c1"># won&#39;t be empty, so inject at least one empty accumulator</span>
<span class="c1"># so that downstream is guaranteed to produce non-empty output.</span>
<span class="n">empty_bundle</span> <span class="o">=</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">core</span><span class="o">.</span><span class="n">Create</span><span class="p">([(</span><span class="kc">None</span><span class="p">,</span> <span class="p">[])])</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span>
<span class="n">top_per_bundle</span><span class="o">.</span><span class="n">element_type</span><span class="p">))</span>
<span class="k">return</span> <span class="p">((</span><span class="n">top_per_bundle</span><span class="p">,</span> <span class="n">empty_bundle</span><span class="p">)</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">Flatten</span><span class="p">()</span>
<span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">GroupByKey</span><span class="p">()</span>
<span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span>
<span class="n">_MergeTopPerBundle</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reverse</span><span class="p">)))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span>
<span class="n">TopCombineFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reverse</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">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span>
<span class="n">TopCombineFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_reverse</span><span class="p">))</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div></div>
<div class="viewcode-block" id="Top.PerKey"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.PerKey">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Tuple</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">@with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">V</span><span class="p">]])</span>
<span class="k">class</span> <span class="nc">PerKey</span><span class="p">(</span><span class="n">ptransform</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Identifies the compare-most N elements associated with each key.</span>
<span class="sd"> This transform will produce a PCollection mapping unique keys in the input</span>
<span class="sd"> PCollection to the n greatest elements with which they are associated, where</span>
<span class="sd"> &quot;greatest&quot; is determined by the comparator function supplied as the compare</span>
<span class="sd"> argument in the initializer.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Creates a per-key Top operation.</span>
<span class="sd"> The arguments &#39;key&#39; and &#39;reverse&#39; may be passed as keyword arguments,</span>
<span class="sd"> and have the same meaning as for Python&#39;s sort functions.</span>
<span class="sd"> Args:</span>
<span class="sd"> n: number of elements to extract from pcoll.</span>
<span class="sd"> key: (optional) a mapping of elements to a comparable key, similar to</span>
<span class="sd"> the key argument of Python&#39;s sorting methods.</span>
<span class="sd"> reverse: (optional) whether to order things smallest to largest, rather</span>
<span class="sd"> than largest to smallest</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_n</span> <span class="o">=</span> <span class="n">n</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_key</span> <span class="o">=</span> <span class="n">key</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_reverse</span> <span class="o">=</span> <span class="n">reverse</span>
<div class="viewcode-block" id="Top.PerKey.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.PerKey.default_label">[docs]</a> <span class="k">def</span> <span class="nf">default_label</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;TopPerKey(</span><span class="si">%d</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span></div>
<div class="viewcode-block" id="Top.PerKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.PerKey.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="w"> </span><span class="sd">&quot;&quot;&quot;Expands the transform.</span>
<span class="sd"> Raises TypeCheckError: If the output type of the input PCollection is not</span>
<span class="sd"> compatible with Tuple[A, B].</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll: PCollection to process</span>
<span class="sd"> Returns:</span>
<span class="sd"> the PCollection containing the result.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombinePerKey</span><span class="p">(</span>
<span class="n">TopCombineFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reverse</span><span class="p">))</span></div></div>
<div class="viewcode-block" id="Top.Largest"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.Largest">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@ptransform</span><span class="o">.</span><span class="n">ptransform_fn</span>
<span class="k">def</span> <span class="nf">Largest</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">has_defaults</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Obtain a list of the greatest N elements in a PCollection.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Top</span><span class="o">.</span><span class="n">Of</span><span class="p">(</span><span class="n">n</span><span class="p">,</span> <span class="n">key</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">Top</span><span class="o">.</span><span class="n">Of</span><span class="p">(</span><span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div>
<div class="viewcode-block" id="Top.Smallest"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.Smallest">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@ptransform</span><span class="o">.</span><span class="n">ptransform_fn</span>
<span class="k">def</span> <span class="nf">Smallest</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">has_defaults</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Obtain a list of the least N elements in a PCollection.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Top</span><span class="o">.</span><span class="n">Of</span><span class="p">(</span><span class="n">n</span><span class="p">,</span> <span class="n">key</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="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Top</span><span class="o">.</span><span class="n">Of</span><span class="p">(</span><span class="n">n</span><span class="p">,</span> <span class="n">key</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="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div>
<div class="viewcode-block" id="Top.LargestPerKey"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.LargestPerKey">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@ptransform</span><span class="o">.</span><span class="n">ptransform_fn</span>
<span class="k">def</span> <span class="nf">LargestPerKey</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Identifies the N greatest elements associated with each key.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Top</span><span class="o">.</span><span class="n">PerKey</span><span class="p">(</span><span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span></div>
<div class="viewcode-block" id="Top.SmallestPerKey"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Top.SmallestPerKey">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@ptransform</span><span class="o">.</span><span class="n">ptransform_fn</span>
<span class="k">def</span> <span class="nf">SmallestPerKey</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="o">*</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Identifies the N least elements associated with each key.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Top</span><span class="o">.</span><span class="n">PerKey</span><span class="p">(</span><span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div></div>
<span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="kc">None</span><span class="p">,</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">_TopPerBundle</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">reverse</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_n</span> <span class="o">=</span> <span class="n">n</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="o">=</span> <span class="n">operator</span><span class="o">.</span><span class="n">gt</span> <span class="k">if</span> <span class="n">reverse</span> <span class="k">else</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_key</span> <span class="o">=</span> <span class="n">key</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">_heap</span> <span class="o">=</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="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">:</span>
<span class="n">element</span> <span class="o">=</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">ComparableValue</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">_compare</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</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">_heap</span><span class="p">)</span> <span class="o">&lt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">:</span>
<span class="n">heapq</span><span class="o">.</span><span class="n">heappush</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_heap</span><span class="p">,</span> <span class="n">element</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">heapq</span><span class="o">.</span><span class="n">heappushpop</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_heap</span><span class="p">,</span> <span class="n">element</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="c1"># Though sorting here results in more total work, this allows us to</span>
<span class="c1"># skip most elements in the reducer.</span>
<span class="c1"># Essentially, given s map bundles, we are trading about O(sn) compares in</span>
<span class="c1"># the (single) reducer for O(sn log n) compares across all mappers.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_heap</span><span class="o">.</span><span class="n">sort</span><span class="p">()</span>
<span class="c1"># Unwrap to avoid serialization via pickle.</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</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="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="p">[</span><span class="n">wrapper</span><span class="o">.</span><span class="n">value</span> <span class="k">for</span> <span class="n">wrapper</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_heap</span><span class="p">]))</span>
<span class="k">else</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="kc">None</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_heap</span><span class="p">))</span>
<span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="kc">None</span><span class="p">,</span> <span class="n">Iterable</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">T</span><span class="p">]]])</span>
<span class="nd">@with_output_types</span><span class="p">(</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">_MergeTopPerBundle</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">reverse</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_n</span> <span class="o">=</span> <span class="n">n</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="o">=</span> <span class="n">operator</span><span class="o">.</span><span class="n">gt</span> <span class="k">if</span> <span class="n">reverse</span> <span class="k">else</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_key</span> <span class="o">=</span> <span class="n">key</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">key_and_bundles</span><span class="p">):</span>
<span class="n">_</span><span class="p">,</span> <span class="n">bundles</span> <span class="o">=</span> <span class="n">key_and_bundles</span>
<span class="k">def</span> <span class="nf">push</span><span class="p">(</span><span class="n">hp</span><span class="p">,</span> <span class="n">e</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">hp</span><span class="p">)</span> <span class="o">&lt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">:</span>
<span class="n">heapq</span><span class="o">.</span><span class="n">heappush</span><span class="p">(</span><span class="n">hp</span><span class="p">,</span> <span class="n">e</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">elif</span> <span class="n">e</span> <span class="o">&lt;</span> <span class="n">hp</span><span class="p">[</span><span class="mi">0</span><span class="p">]:</span>
<span class="c1"># Because _TopPerBundle returns sorted lists, all other elements</span>
<span class="c1"># will also be smaller.</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">heapq</span><span class="o">.</span><span class="n">heappushpop</span><span class="p">(</span><span class="n">hp</span><span class="p">,</span> <span class="n">e</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">:</span>
<span class="n">heapc</span> <span class="o">=</span> <span class="p">[]</span> <span class="c1"># type: List[cy_combiners.ComparableValue]</span>
<span class="k">for</span> <span class="n">bundle</span> <span class="ow">in</span> <span class="n">bundles</span><span class="p">:</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">heapc</span><span class="p">:</span>
<span class="n">heapc</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">cy_combiners</span><span class="o">.</span><span class="n">ComparableValue</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">_compare</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">)</span>
<span class="k">for</span> <span class="n">element</span> <span class="ow">in</span> <span class="n">bundle</span>
<span class="p">]</span>
<span class="k">continue</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/21205): Remove this</span>
<span class="c1"># workaround once legacy dataflow correctly handles coders with</span>
<span class="c1"># combiner packing and/or is deprecated.</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">bundle</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="n">bundle</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">bundle</span><span class="p">)</span>
<span class="k">for</span> <span class="n">element</span> <span class="ow">in</span> <span class="nb">reversed</span><span class="p">(</span><span class="n">bundle</span><span class="p">):</span>
<span class="k">if</span> <span class="n">push</span><span class="p">(</span><span class="n">heapc</span><span class="p">,</span>
<span class="n">cy_combiners</span><span class="o">.</span><span class="n">ComparableValue</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">_compare</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">)):</span>
<span class="k">break</span>
<span class="n">heapc</span><span class="o">.</span><span class="n">sort</span><span class="p">()</span>
<span class="k">yield</span> <span class="p">[</span><span class="n">wrapper</span><span class="o">.</span><span class="n">value</span> <span class="k">for</span> <span class="n">wrapper</span> <span class="ow">in</span> <span class="nb">reversed</span><span class="p">(</span><span class="n">heapc</span><span class="p">)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">heap</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">bundle</span> <span class="ow">in</span> <span class="n">bundles</span><span class="p">:</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/21205): Remove this</span>
<span class="c1"># workaround once legacy dataflow correctly handles coders with</span>
<span class="c1"># combiner packing and/or is deprecated.</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">bundle</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="n">bundle</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">bundle</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">heap</span><span class="p">:</span>
<span class="n">heap</span> <span class="o">=</span> <span class="n">bundle</span>
<span class="k">continue</span>
<span class="k">for</span> <span class="n">element</span> <span class="ow">in</span> <span class="nb">reversed</span><span class="p">(</span><span class="n">bundle</span><span class="p">):</span>
<span class="k">if</span> <span class="n">push</span><span class="p">(</span><span class="n">heap</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="k">break</span>
<span class="n">heap</span><span class="o">.</span><span class="n">sort</span><span class="p">()</span>
<span class="k">yield</span> <span class="n">heap</span><span class="p">[::</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span>
<div class="viewcode-block" id="TopCombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.TopCombineFn">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</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">TopCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;CombineFn doing the combining for all of the Top transforms.</span>
<span class="sd"> This CombineFn uses a key or comparison operator to rank the elements.</span>
<span class="sd"> Args:</span>
<span class="sd"> key: (optional) a mapping of elements to a comparable key, similar to</span>
<span class="sd"> the key argument of Python&#39;s sorting methods.</span>
<span class="sd"> reverse: (optional) whether to order things smallest to largest, rather</span>
<span class="sd"> than largest to smallest</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_n</span> <span class="o">=</span> <span class="n">n</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="o">=</span> <span class="n">operator</span><span class="o">.</span><span class="n">gt</span> <span class="k">if</span> <span class="n">reverse</span> <span class="k">else</span> <span class="n">operator</span><span class="o">.</span><span class="n">lt</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_key</span> <span class="o">=</span> <span class="n">key</span>
<span class="k">def</span> <span class="nf">_hydrated_heap</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">heap</span><span class="p">):</span>
<span class="k">if</span> <span class="n">heap</span><span class="p">:</span>
<span class="n">first</span> <span class="o">=</span> <span class="n">heap</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">first</span><span class="p">,</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">ComparableValue</span><span class="p">):</span>
<span class="k">if</span> <span class="n">first</span><span class="o">.</span><span class="n">requires_hydration</span><span class="p">:</span>
<span class="k">for</span> <span class="n">comparable</span> <span class="ow">in</span> <span class="n">heap</span><span class="p">:</span>
<span class="k">assert</span> <span class="n">comparable</span><span class="o">.</span><span class="n">requires_hydration</span>
<span class="n">comparable</span><span class="o">.</span><span class="n">hydrate</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_compare</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">)</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">comparable</span><span class="o">.</span><span class="n">requires_hydration</span>
<span class="k">return</span> <span class="n">heap</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">heap</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="p">[</span>
<span class="n">cy_combiners</span><span class="o">.</span><span class="n">ComparableValue</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">_compare</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">)</span>
<span class="k">for</span> <span class="n">element</span> <span class="ow">in</span> <span class="n">heap</span>
<span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">heap</span>
<div class="viewcode-block" id="TopCombineFn.display_data"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.TopCombineFn.display_data">[docs]</a> <span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">{</span>
<span class="s1">&#39;n&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">,</span>
<span class="s1">&#39;compare&#39;</span><span class="p">:</span> <span class="n">DisplayDataItem</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_compare</span><span class="o">.</span><span class="vm">__name__</span> <span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_compare</span><span class="p">,</span> <span class="s1">&#39;__name__&#39;</span><span class="p">)</span> <span class="k">else</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_compare</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span><span class="p">)</span><span class="o">.</span><span class="n">drop_if_none</span><span class="p">()</span>
<span class="p">}</span></div>
<span class="c1"># The accumulator type is a tuple</span>
<span class="c1"># (bool, Union[List[T], List[ComparableValue[T]])</span>
<span class="c1"># where the boolean indicates whether the second slot contains a List of T</span>
<span class="c1"># (False) or List of ComparableValue[T] (True). In either case, the List</span>
<span class="c1"># maintains heap invariance. When the contents of the List are</span>
<span class="c1"># ComparableValue[T] they either all &#39;requires_hydration&#39; or none do.</span>
<span class="c1"># This accumulator representation allows us to minimize the data encoding</span>
<span class="c1"># overheads. Creation of ComparableValues is elided for performance reasons</span>
<span class="c1"># when there is no need for complicated comparison functions.</span>
<div class="viewcode-block" id="TopCombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.TopCombineFn.create_accumulator">[docs]</a> <span class="k">def</span> <span class="nf">create_accumulator</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span><span class="kc">False</span><span class="p">,</span> <span class="p">[])</span></div>
<div class="viewcode-block" id="TopCombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.TopCombineFn.add_input">[docs]</a> <span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="c1"># Caching to avoid paying the price of variadic expansion of args / kwargs</span>
<span class="c1"># when it&#39;s not needed (for the &#39;if&#39; case below).</span>
<span class="n">holds_comparables</span><span class="p">,</span> <span class="n">heap</span> <span class="o">=</span> <span class="n">accumulator</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">operator</span><span class="o">.</span><span class="n">lt</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">:</span>
<span class="n">heap</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_hydrated_heap</span><span class="p">(</span><span class="n">heap</span><span class="p">)</span>
<span class="n">holds_comparables</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">holds_comparables</span>
<span class="n">comparable</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">cy_combiners</span><span class="o">.</span><span class="n">ComparableValue</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">_compare</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">)</span>
<span class="k">if</span> <span class="n">holds_comparables</span> <span class="k">else</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="n">heap</span><span class="p">)</span> <span class="o">&lt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">:</span>
<span class="n">heapq</span><span class="o">.</span><span class="n">heappush</span><span class="p">(</span><span class="n">heap</span><span class="p">,</span> <span class="n">comparable</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">heapq</span><span class="o">.</span><span class="n">heappushpop</span><span class="p">(</span><span class="n">heap</span><span class="p">,</span> <span class="n">comparable</span><span class="p">)</span>
<span class="k">return</span> <span class="p">(</span><span class="n">holds_comparables</span><span class="p">,</span> <span class="n">heap</span><span class="p">)</span></div>
<div class="viewcode-block" id="TopCombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.TopCombineFn.merge_accumulators">[docs]</a> <span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="n">result_heap</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">holds_comparables</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">for</span> <span class="n">accumulator</span> <span class="ow">in</span> <span class="n">accumulators</span><span class="p">:</span>
<span class="n">holds_comparables</span><span class="p">,</span> <span class="n">heap</span> <span class="o">=</span> <span class="n">accumulator</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">operator</span><span class="o">.</span><span class="n">lt</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">:</span>
<span class="n">heap</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_hydrated_heap</span><span class="p">(</span><span class="n">heap</span><span class="p">)</span>
<span class="n">holds_comparables</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">holds_comparables</span>
<span class="k">if</span> <span class="n">result_heap</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">result_heap</span> <span class="o">=</span> <span class="n">heap</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">for</span> <span class="n">comparable</span> <span class="ow">in</span> <span class="n">heap</span><span class="p">:</span>
<span class="n">_</span><span class="p">,</span> <span class="n">result_heap</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">add_input</span><span class="p">(</span>
<span class="p">(</span><span class="n">holds_comparables</span><span class="p">,</span> <span class="n">result_heap</span><span class="p">),</span>
<span class="n">comparable</span><span class="o">.</span><span class="n">value</span> <span class="k">if</span> <span class="n">holds_comparables</span> <span class="k">else</span> <span class="n">comparable</span><span class="p">)</span>
<span class="k">assert</span> <span class="n">result_heap</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">holds_comparables</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="k">return</span> <span class="p">(</span><span class="n">holds_comparables</span><span class="p">,</span> <span class="n">result_heap</span><span class="p">)</span></div>
<div class="viewcode-block" id="TopCombineFn.compact"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.TopCombineFn.compact">[docs]</a> <span class="k">def</span> <span class="nf">compact</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="n">holds_comparables</span><span class="p">,</span> <span class="n">heap</span> <span class="o">=</span> <span class="n">accumulator</span>
<span class="c1"># Unwrap to avoid serialization via pickle.</span>
<span class="k">if</span> <span class="n">holds_comparables</span><span class="p">:</span>
<span class="k">return</span> <span class="p">(</span><span class="kc">False</span><span class="p">,</span> <span class="p">[</span><span class="n">comparable</span><span class="o">.</span><span class="n">value</span> <span class="k">for</span> <span class="n">comparable</span> <span class="ow">in</span> <span class="n">heap</span><span class="p">])</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">accumulator</span></div>
<div class="viewcode-block" id="TopCombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.TopCombineFn.extract_output">[docs]</a> <span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="n">holds_comparables</span><span class="p">,</span> <span class="n">heap</span> <span class="o">=</span> <span class="n">accumulator</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_compare</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">operator</span><span class="o">.</span><span class="n">lt</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key</span><span class="p">:</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">holds_comparables</span><span class="p">:</span>
<span class="n">heap</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_hydrated_heap</span><span class="p">(</span><span class="n">heap</span><span class="p">)</span>
<span class="n">holds_comparables</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">holds_comparables</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">heap</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span>
<span class="n">heap</span><span class="o">.</span><span class="n">sort</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="k">return</span> <span class="p">[</span>
<span class="n">comparable</span><span class="o">.</span><span class="n">value</span> <span class="k">if</span> <span class="n">holds_comparables</span> <span class="k">else</span> <span class="n">comparable</span>
<span class="k">for</span> <span class="n">comparable</span> <span class="ow">in</span> <span class="n">heap</span>
<span class="p">]</span></div></div>
<span class="k">class</span> <span class="nc">Largest</span><span class="p">(</span><span class="n">TopCombineFn</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">default_label</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;Largest(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span>
<span class="k">class</span> <span class="nc">Smallest</span><span class="p">(</span><span class="n">TopCombineFn</span><span class="p">):</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">):</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="n">n</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="k">def</span> <span class="nf">default_label</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;Smallest(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span>
<div class="viewcode-block" id="Sample"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample">[docs]</a><span class="k">class</span> <span class="nc">Sample</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Combiners for sampling n elements without replacement.&quot;&quot;&quot;</span>
<span class="c1"># pylint: disable=no-self-argument</span>
<div class="viewcode-block" id="Sample.FixedSizeGlobally"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample.FixedSizeGlobally">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</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">FixedSizeGlobally</span><span class="p">(</span><span class="n">CombinerWithoutDefaults</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Sample n elements from the input PCollection without replacement.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">):</span>
<span class="nb">super</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">_n</span> <span class="o">=</span> <span class="n">n</span>
<div class="viewcode-block" id="Sample.FixedSizeGlobally.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample.FixedSizeGlobally.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="bp">self</span><span class="o">.</span><span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">SampleCombineFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</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">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">SampleCombineFn</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">))</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div>
<div class="viewcode-block" id="Sample.FixedSizeGlobally.display_data"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample.FixedSizeGlobally.display_data">[docs]</a> <span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">{</span><span class="s1">&#39;n&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">}</span></div>
<div class="viewcode-block" id="Sample.FixedSizeGlobally.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample.FixedSizeGlobally.default_label">[docs]</a> <span class="k">def</span> <span class="nf">default_label</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;FixedSizeGlobally(</span><span class="si">%d</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span></div></div>
<div class="viewcode-block" id="Sample.FixedSizePerKey"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample.FixedSizePerKey">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Tuple</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">@with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">V</span><span class="p">]])</span>
<span class="k">class</span> <span class="nc">FixedSizePerKey</span><span class="p">(</span><span class="n">ptransform</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Sample n elements associated with each key without replacement.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_n</span> <span class="o">=</span> <span class="n">n</span>
<div class="viewcode-block" id="Sample.FixedSizePerKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample.FixedSizePerKey.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="n">pcoll</span> <span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombinePerKey</span><span class="p">(</span><span class="n">SampleCombineFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">))</span></div>
<div class="viewcode-block" id="Sample.FixedSizePerKey.display_data"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample.FixedSizePerKey.display_data">[docs]</a> <span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">{</span><span class="s1">&#39;n&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">}</span></div>
<div class="viewcode-block" id="Sample.FixedSizePerKey.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Sample.FixedSizePerKey.default_label">[docs]</a> <span class="k">def</span> <span class="nf">default_label</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;FixedSizePerKey(</span><span class="si">%d</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_n</span></div></div></div>
<div class="viewcode-block" id="SampleCombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.SampleCombineFn">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</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">SampleCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;CombineFn for all Sample transforms.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">):</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="c1"># Most of this combiner&#39;s work is done by a TopCombineFn. We could just</span>
<span class="c1"># subclass TopCombineFn to make this class, but since sampling is not</span>
<span class="c1"># really a kind of Top operation, we use a TopCombineFn instance as a</span>
<span class="c1"># helper instead.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_top_combiner</span> <span class="o">=</span> <span class="n">TopCombineFn</span><span class="p">(</span><span class="n">n</span><span class="p">)</span>
<div class="viewcode-block" id="SampleCombineFn.setup"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.SampleCombineFn.setup">[docs]</a> <span class="k">def</span> <span class="nf">setup</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">_top_combiner</span><span class="o">.</span><span class="n">setup</span><span class="p">()</span></div>
<div class="viewcode-block" id="SampleCombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.SampleCombineFn.create_accumulator">[docs]</a> <span class="k">def</span> <span class="nf">create_accumulator</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">_top_combiner</span><span class="o">.</span><span class="n">create_accumulator</span><span class="p">()</span></div>
<div class="viewcode-block" id="SampleCombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.SampleCombineFn.add_input">[docs]</a> <span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">heap</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="c1"># Before passing elements to the Top combiner, we pair them with random</span>
<span class="c1"># numbers. The elements with the n largest random number &quot;keys&quot; will be</span>
<span class="c1"># selected for the output.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_top_combiner</span><span class="o">.</span><span class="n">add_input</span><span class="p">(</span><span class="n">heap</span><span class="p">,</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="n">element</span><span class="p">))</span></div>
<div class="viewcode-block" id="SampleCombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.SampleCombineFn.merge_accumulators">[docs]</a> <span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">heaps</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_top_combiner</span><span class="o">.</span><span class="n">merge_accumulators</span><span class="p">(</span><span class="n">heaps</span><span class="p">)</span></div>
<div class="viewcode-block" id="SampleCombineFn.compact"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.SampleCombineFn.compact">[docs]</a> <span class="k">def</span> <span class="nf">compact</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">heap</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_top_combiner</span><span class="o">.</span><span class="n">compact</span><span class="p">(</span><span class="n">heap</span><span class="p">)</span></div>
<div class="viewcode-block" id="SampleCombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.SampleCombineFn.extract_output">[docs]</a> <span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">heap</span><span class="p">):</span>
<span class="c1"># Here we strip off the random number keys we added in add_input.</span>
<span class="k">return</span> <span class="p">[</span><span class="n">e</span> <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">e</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_top_combiner</span><span class="o">.</span><span class="n">extract_output</span><span class="p">(</span><span class="n">heap</span><span class="p">)]</span></div>
<div class="viewcode-block" id="SampleCombineFn.teardown"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.SampleCombineFn.teardown">[docs]</a> <span class="k">def</span> <span class="nf">teardown</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">_top_combiner</span><span class="o">.</span><span class="n">teardown</span><span class="p">()</span></div></div>
<span class="k">class</span> <span class="nc">_TupleCombineFnBase</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">combiners</span><span class="p">,</span> <span class="n">merge_accumulators_batch_size</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span> <span class="o">=</span> <span class="p">[</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="o">.</span><span class="n">maybe_from_callable</span><span class="p">(</span><span class="n">c</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">combiners</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_named_combiners</span> <span class="o">=</span> <span class="n">combiners</span>
<span class="c1"># If the `merge_accumulators_batch_size` value is not specified, we chose a</span>
<span class="c1"># bounded default that is inversely proportional to the number of</span>
<span class="c1"># accumulators in merged tuples.</span>
<span class="n">num_combiners</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="nb">len</span><span class="p">(</span><span class="n">combiners</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_merge_accumulators_batch_size</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">merge_accumulators_batch_size</span> <span class="ow">or</span> <span class="nb">max</span><span class="p">(</span><span class="mi">10</span><span class="p">,</span> <span class="mi">1000</span> <span class="o">//</span> <span class="n">num_combiners</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">combiners</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">c</span><span class="o">.</span><span class="vm">__name__</span> <span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">c</span><span class="p">,</span> <span class="s1">&#39;__name__&#39;</span><span class="p">)</span> <span class="k">else</span> <span class="n">c</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span>
<span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_named_combiners</span>
<span class="p">]</span>
<span class="k">return</span> <span class="p">{</span>
<span class="s1">&#39;combiners&#39;</span><span class="p">:</span> <span class="nb">str</span><span class="p">(</span><span class="n">combiners</span><span class="p">),</span>
<span class="s1">&#39;merge_accumulators_batch_size&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_merge_accumulators_batch_size</span>
<span class="p">}</span>
<span class="k">def</span> <span class="nf">setup</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">:</span>
<span class="n">c</span><span class="o">.</span><span class="n">setup</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">create_accumulator</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span><span class="n">c</span><span class="o">.</span><span class="n">create_accumulator</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="c1"># Make sure that `accumulators` is an iterator (so that the position is</span>
<span class="c1"># remembered).</span>
<span class="n">accumulators</span> <span class="o">=</span> <span class="nb">iter</span><span class="p">(</span><span class="n">accumulators</span><span class="p">)</span>
<span class="n">result</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">accumulators</span><span class="p">)</span>
<span class="k">while</span> <span class="kc">True</span><span class="p">:</span>
<span class="c1"># Load accumulators into memory and merge in batches to decrease peak</span>
<span class="c1"># memory usage.</span>
<span class="n">accumulators_batch</span> <span class="o">=</span> <span class="p">[</span><span class="n">result</span><span class="p">]</span> <span class="o">+</span> <span class="nb">list</span><span class="p">(</span>
<span class="n">itertools</span><span class="o">.</span><span class="n">islice</span><span class="p">(</span><span class="n">accumulators</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_merge_accumulators_batch_size</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">accumulators_batch</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">break</span>
<span class="n">result</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">c</span><span class="o">.</span><span class="n">merge_accumulators</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span><span class="p">,</span>
<span class="n">a</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">,</span> <span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">accumulators_batch</span><span class="p">))</span>
<span class="p">]</span>
<span class="k">return</span> <span class="n">result</span>
<span class="k">def</span> <span class="nf">compact</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span>
<span class="n">c</span><span class="o">.</span><span class="n">compact</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span><span class="p">,</span>
<span class="n">a</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">)</span>
<span class="p">]</span>
<span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">tuple</span><span class="p">(</span>
<span class="n">c</span><span class="o">.</span><span class="n">extract_output</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span><span class="p">,</span>
<span class="n">a</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">teardown</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="nb">reversed</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">):</span>
<span class="n">c</span><span class="o">.</span><span class="n">teardown</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">TupleCombineFn</span><span class="p">(</span><span class="n">_TupleCombineFnBase</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A combiner for combining tuples via a tuple of combiners.</span>
<span class="sd"> Takes as input a tuple of N CombineFns and combines N-tuples by</span>
<span class="sd"> combining the k-th element of each tuple with the k-th CombineFn,</span>
<span class="sd"> outputting a new N-tuple of combined values.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span>
<span class="n">c</span><span class="o">.</span><span class="n">add_input</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">e</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span><span class="p">,</span>
<span class="n">a</span><span class="p">,</span>
<span class="n">e</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">)</span>
<span class="p">]</span>
<span class="k">def</span> <span class="nf">with_common_input</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">SingleInputTupleCombineFn</span><span class="p">(</span><span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">SingleInputTupleCombineFn</span><span class="p">(</span><span class="n">_TupleCombineFnBase</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A combiner for combining a single value via a tuple of combiners.</span>
<span class="sd"> Takes as input a tuple of N CombineFns and combines elements by</span>
<span class="sd"> applying each CombineFn to each input, producing an N-tuple of</span>
<span class="sd"> the outputs corresponding to each of the N CombineFn&#39;s outputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span>
<span class="n">c</span><span class="o">.</span><span class="n">add_input</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span><span class="p">,</span>
<span class="n">a</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_combiners</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">)</span>
<span class="p">]</span>
<div class="viewcode-block" id="ToList"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToList">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</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">ToList</span><span class="p">(</span><span class="n">CombinerWithoutDefaults</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A global CombineFn that condenses a PCollection into a single list.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="ToList.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToList.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="bp">self</span><span class="o">.</span><span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">ToListCombineFn</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="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span>
<span class="n">ToListCombineFn</span><span class="p">())</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div></div>
<div class="viewcode-block" id="ToListCombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToListCombineFn">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</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">ToListCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;CombineFn for to_list.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="ToListCombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToListCombineFn.create_accumulator">[docs]</a> <span class="k">def</span> <span class="nf">create_accumulator</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[]</span></div>
<div class="viewcode-block" id="ToListCombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToListCombineFn.add_input">[docs]</a> <span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="n">accumulator</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">return</span> <span class="n">accumulator</span></div>
<div class="viewcode-block" id="ToListCombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToListCombineFn.merge_accumulators">[docs]</a> <span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">sum</span><span class="p">(</span><span class="n">accumulators</span><span class="p">,</span> <span class="p">[])</span></div>
<div class="viewcode-block" id="ToListCombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToListCombineFn.extract_output">[docs]</a> <span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">accumulator</span></div></div>
<div class="viewcode-block" id="ToDict"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToDict">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Tuple</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">@with_output_types</span><span class="p">(</span><span class="n">Dict</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">ToDict</span><span class="p">(</span><span class="n">CombinerWithoutDefaults</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A global CombineFn that condenses a PCollection into a single dict.</span>
<span class="sd"> PCollections should consist of 2-tuples, notionally (key, value) pairs.</span>
<span class="sd"> If multiple values are associated with the same key, only one of the values</span>
<span class="sd"> will be present in the resulting dict.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="ToDict.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToDict.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="bp">self</span><span class="o">.</span><span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">ToDictCombineFn</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="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span>
<span class="n">ToDictCombineFn</span><span class="p">())</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div></div>
<div class="viewcode-block" id="ToDictCombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToDictCombineFn">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Tuple</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">@with_output_types</span><span class="p">(</span><span class="n">Dict</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">ToDictCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;CombineFn for to_dict.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="ToDictCombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToDictCombineFn.create_accumulator">[docs]</a> <span class="k">def</span> <span class="nf">create_accumulator</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">{}</span></div>
<div class="viewcode-block" id="ToDictCombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToDictCombineFn.add_input">[docs]</a> <span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</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">value</span> <span class="o">=</span> <span class="n">element</span>
<span class="n">accumulator</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span>
<span class="k">return</span> <span class="n">accumulator</span></div>
<div class="viewcode-block" id="ToDictCombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToDictCombineFn.merge_accumulators">[docs]</a> <span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">):</span>
<span class="n">result</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">a</span> <span class="ow">in</span> <span class="n">accumulators</span><span class="p">:</span>
<span class="n">result</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">a</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="ToDictCombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToDictCombineFn.extract_output">[docs]</a> <span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">accumulator</span></div></div>
<div class="viewcode-block" id="ToSet"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToSet">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</span><span class="n">Set</span><span class="p">[</span><span class="n">T</span><span class="p">])</span>
<span class="k">class</span> <span class="nc">ToSet</span><span class="p">(</span><span class="n">CombinerWithoutDefaults</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A global CombineFn that condenses a PCollection into a set.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="ToSet.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToSet.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="bp">self</span><span class="o">.</span><span class="n">has_defaults</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">ToSetCombineFn</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="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span>
<span class="n">ToSetCombineFn</span><span class="p">())</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">()</span></div></div>
<div class="viewcode-block" id="ToSetCombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToSetCombineFn">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@with_output_types</span><span class="p">(</span><span class="n">Set</span><span class="p">[</span><span class="n">T</span><span class="p">])</span>
<span class="k">class</span> <span class="nc">ToSetCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;CombineFn for ToSet.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="ToSetCombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToSetCombineFn.create_accumulator">[docs]</a> <span class="k">def</span> <span class="nf">create_accumulator</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">set</span><span class="p">()</span></div>
<div class="viewcode-block" id="ToSetCombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToSetCombineFn.add_input">[docs]</a> <span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="n">accumulator</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">element</span><span class="p">)</span>
<span class="k">return</span> <span class="n">accumulator</span></div>
<div class="viewcode-block" id="ToSetCombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToSetCombineFn.merge_accumulators">[docs]</a> <span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">set</span><span class="o">.</span><span class="n">union</span><span class="p">(</span><span class="o">*</span><span class="n">accumulators</span><span class="p">)</span></div>
<div class="viewcode-block" id="ToSetCombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.ToSetCombineFn.extract_output">[docs]</a> <span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">accumulator</span></div></div>
<span class="k">class</span> <span class="nc">_CurriedFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Wrapped CombineFn with extra arguments.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">fn</span><span class="p">,</span> <span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">fn</span> <span class="o">=</span> <span class="n">fn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">args</span> <span class="o">=</span> <span class="n">args</span>
<span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span> <span class="o">=</span> <span class="n">kwargs</span>
<span class="k">def</span> <span class="nf">setup</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">fn</span><span class="o">.</span><span class="n">setup</span><span class="p">(</span><span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">create_accumulator</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">fn</span><span class="o">.</span><span class="n">create_accumulator</span><span class="p">(</span><span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">add_input</span><span class="p">(</span><span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">merge_accumulators</span><span class="p">(</span><span class="n">accumulators</span><span class="p">,</span> <span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">compact</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">compact</span><span class="p">(</span><span class="n">accumulator</span><span class="p">,</span> <span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">extract_output</span><span class="p">(</span><span class="n">accumulator</span><span class="p">,</span> <span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">teardown</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">fn</span><span class="o">.</span><span class="n">teardown</span><span class="p">(</span><span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">apply</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">elements</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">elements</span><span class="p">,</span> <span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">curry_combine_fn</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">args</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">kwargs</span><span class="p">:</span>
<span class="k">return</span> <span class="n">fn</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">_CurriedFn</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">PhasedCombineFnExecutor</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Executor for phases of combine operations.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">phase</span><span class="p">,</span> <span class="n">fn</span><span class="p">,</span> <span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span> <span class="o">=</span> <span class="n">curry_combine_fn</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span><span class="p">)</span>
<span class="k">if</span> <span class="n">phase</span> <span class="o">==</span> <span class="s1">&#39;all&#39;</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">apply</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">full_combine</span>
<span class="k">elif</span> <span class="n">phase</span> <span class="o">==</span> <span class="s1">&#39;add&#39;</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">apply</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">add_only</span>
<span class="k">elif</span> <span class="n">phase</span> <span class="o">==</span> <span class="s1">&#39;merge&#39;</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">apply</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">merge_only</span>
<span class="k">elif</span> <span class="n">phase</span> <span class="o">==</span> <span class="s1">&#39;extract&#39;</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">apply</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">extract_only</span>
<span class="k">elif</span> <span class="n">phase</span> <span class="o">==</span> <span class="s1">&#39;convert&#39;</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">apply</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">convert_to_accumulator</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Unexpected phase: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">phase</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">full_combine</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">elements</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">elements</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">add_only</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">elements</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">add_inputs</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">create_accumulator</span><span class="p">(),</span> <span class="n">elements</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">merge_only</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">merge_accumulators</span><span class="p">(</span><span class="n">accumulators</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">extract_only</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">extract_output</span><span class="p">(</span><span class="n">accumulator</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">convert_to_accumulator</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="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">add_input</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">create_accumulator</span><span class="p">(),</span> <span class="n">element</span><span class="p">)</span>
<div class="viewcode-block" id="Latest"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Latest">[docs]</a><span class="k">class</span> <span class="nc">Latest</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Combiners for computing the latest element&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Latest.Globally"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Latest.Globally">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@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">Globally</span><span class="p">(</span><span class="n">CombinerWithoutDefaults</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Compute the element with the latest timestamp from a</span>
<span class="sd"> PCollection.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Latest.Globally.add_timestamp"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Latest.Globally.add_timestamp">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">add_timestamp</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">core</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="k">return</span> <span class="p">[(</span><span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)]</span></div>
<div class="viewcode-block" id="Latest.Globally.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Latest.Globally.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="bp">self</span><span class="o">.</span><span class="n">has_defaults</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="n">core</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">add_timestamp</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span>
<span class="n">Tuple</span><span class="p">[</span><span class="n">T</span><span class="p">,</span> <span class="n">TimestampType</span><span class="p">])</span>
<span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">LatestCombineFn</span><span class="p">()))</span>
<span class="k">else</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="n">core</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">add_timestamp</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span>
<span class="n">Tuple</span><span class="p">[</span><span class="n">T</span><span class="p">,</span> <span class="n">TimestampType</span><span class="p">])</span>
<span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="n">LatestCombineFn</span><span class="p">())</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">())</span></div></div>
<div class="viewcode-block" id="Latest.PerKey"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Latest.PerKey">[docs]</a> <span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Tuple</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">@with_output_types</span><span class="p">(</span><span class="n">Tuple</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">PerKey</span><span class="p">(</span><span class="n">ptransform</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Compute elements with the latest timestamp for each key</span>
<span class="sd"> from a keyed PCollection&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Latest.PerKey.add_timestamp"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Latest.PerKey.add_timestamp">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">add_timestamp</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">core</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="p">[(</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></div>
<div class="viewcode-block" id="Latest.PerKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.Latest.PerKey.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="n">core</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">add_timestamp</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span>
<span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">Tuple</span><span class="p">[</span><span class="n">T</span><span class="p">,</span> <span class="n">TimestampType</span><span class="p">]])</span>
<span class="o">|</span> <span class="n">core</span><span class="o">.</span><span class="n">CombinePerKey</span><span class="p">(</span><span class="n">LatestCombineFn</span><span class="p">()))</span></div></div></div>
<div class="viewcode-block" id="LatestCombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.LatestCombineFn">[docs]</a><span class="nd">@with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">T</span><span class="p">,</span> <span class="n">TimestampType</span><span class="p">])</span>
<span class="nd">@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">LatestCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;CombineFn to get the element with the latest timestamp</span>
<span class="sd"> from a PCollection.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="LatestCombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.LatestCombineFn.create_accumulator">[docs]</a> <span class="k">def</span> <span class="nf">create_accumulator</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="n">window</span><span class="o">.</span><span class="n">MIN_TIMESTAMP</span><span class="p">)</span></div>
<div class="viewcode-block" id="LatestCombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.LatestCombineFn.add_input">[docs]</a> <span class="k">def</span> <span class="nf">add_input</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="k">if</span> <span class="n">accumulator</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span> <span class="o">&gt;</span> <span class="n">element</span><span class="p">[</span><span class="mi">1</span><span class="p">]:</span>
<span class="k">return</span> <span class="n">accumulator</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">element</span></div>
<div class="viewcode-block" id="LatestCombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.LatestCombineFn.merge_accumulators">[docs]</a> <span class="k">def</span> <span class="nf">merge_accumulators</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulators</span><span class="p">):</span>
<span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">create_accumulator</span><span class="p">()</span>
<span class="k">for</span> <span class="n">accumulator</span> <span class="ow">in</span> <span class="n">accumulators</span><span class="p">:</span>
<span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">add_input</span><span class="p">(</span><span class="n">result</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="LatestCombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.combiners.html#apache_beam.transforms.combiners.LatestCombineFn.extract_output">[docs]</a> <span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">accumulator</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span></div></div>
</pre></div>
</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/rtfd/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">
jQuery(function () {
SphinxRtdTheme.Navigation.enable(true);
});
</script>
</body>
</html>