blob: d4bd75097f5297db1614772ad6fcc69d2848c54c [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.transforms.util &mdash; Apache Beam documentation</title>
<link rel="stylesheet" href="../../../_static/css/theme.css" type="text/css" />
<link rel="index" title="Index"
href="../../../genindex.html"/>
<link rel="search" title="Search" href="../../../search.html"/>
<link rel="top" title="Apache Beam documentation" href="../../../index.html"/>
<link rel="up" title="Module code" href="../../index.html"/>
<script src="../../../_static/js/modernizr.min.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search">
<a href="../../../index.html" class="icon icon-home"> Apache Beam
</a>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.internal.html">apache_beam.internal package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.utils.html">apache_beam.utils package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.version.html">apache_beam.version module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../index.html">Docs</a> &raquo;</li>
<li><a href="../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.transforms.util</li>
<li class="wy-breadcrumbs-aside">
</li>
</ul>
<hr/>
</div>
<div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article">
<div itemprop="articleBody">
<h1>Source code for apache_beam.transforms.util</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="sd">&quot;&quot;&quot;Simple utility PTransforms.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">CombinePerKey</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">Flatten</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">GroupByKey</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">Map</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="k">import</span> <span class="n">PTransform</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="k">import</span> <span class="n">ptransform_fn</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span>
<span class="s1">&#39;CoGroupByKey&#39;</span><span class="p">,</span>
<span class="s1">&#39;Keys&#39;</span><span class="p">,</span>
<span class="s1">&#39;KvSwap&#39;</span><span class="p">,</span>
<span class="s1">&#39;RemoveDuplicates&#39;</span><span class="p">,</span>
<span class="s1">&#39;Values&#39;</span><span class="p">,</span>
<span class="p">]</span>
<div class="viewcode-block" id="CoGroupByKey"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.CoGroupByKey">[docs]</a><span class="k">class</span> <span class="nc">CoGroupByKey</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Groups results across several PCollections by key.</span>
<span class="sd"> Given an input dict mapping serializable keys (called &quot;tags&quot;) to 0 or more</span>
<span class="sd"> PCollections of (key, value) tuples, e.g.::</span>
<span class="sd"> {&#39;pc1&#39;: pcoll1, &#39;pc2&#39;: pcoll2, 33333: pcoll3}</span>
<span class="sd"> creates a single output PCollection of (key, value) tuples whose keys are the</span>
<span class="sd"> unique input keys from all inputs, and whose values are dicts mapping each</span>
<span class="sd"> tag to an iterable of whatever values were under the key in the corresponding</span>
<span class="sd"> PCollection::</span>
<span class="sd"> (&#39;some key&#39;, {&#39;pc1&#39;: [&#39;value 1 under &quot;some key&quot; in pcoll1&#39;,</span>
<span class="sd"> &#39;value 2 under &quot;some key&quot; in pcoll1&#39;],</span>
<span class="sd"> &#39;pc2&#39;: [],</span>
<span class="sd"> 33333: [&#39;only value under &quot;some key&quot; in pcoll3&#39;]})</span>
<span class="sd"> Note that pcoll2 had no values associated with &quot;some key&quot;.</span>
<span class="sd"> CoGroupByKey also works for tuples, lists, or other flat iterables of</span>
<span class="sd"> PCollections, in which case the values of the resulting PCollections</span>
<span class="sd"> will be tuples whose nth value is the list of values from the nth</span>
<span class="sd"> PCollection---conceptually, the &quot;tags&quot; are the indices into the input.</span>
<span class="sd"> Thus, for this input::</span>
<span class="sd"> (pcoll1, pcoll2, pcoll3)</span>
<span class="sd"> the output PCollection&#39;s value for &quot;some key&quot; is::</span>
<span class="sd"> (&#39;some key&#39;, ([&#39;value 1 under &quot;some key&quot; in pcoll1&#39;,</span>
<span class="sd"> &#39;value 2 under &quot;some key&quot; in pcoll1&#39;],</span>
<span class="sd"> [],</span>
<span class="sd"> [&#39;only value under &quot;some key&quot; in pcoll3&#39;]))</span>
<span class="sd"> Args:</span>
<span class="sd"> label: name of this transform instance. Useful while monitoring and</span>
<span class="sd"> debugging a pipeline execution.</span>
<span class="sd"> **kwargs: Accepts a single named argument &quot;pipeline&quot;, which specifies the</span>
<span class="sd"> pipeline that &quot;owns&quot; this PTransform. Ordinarily CoGroupByKey can obtain</span>
<span class="sd"> this information from one of the input PCollections, but if there are none</span>
<span class="sd"> (or if there&#39;s a chance there may be none), this argument is the only way</span>
<span class="sd"> to provide pipeline information, and should be considered mandatory.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">CoGroupByKey</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="s1">&#39;pipeline&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">if</span> <span class="n">kwargs</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Unexpected keyword arguments: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">_extract_input_pvalues</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalueish</span><span class="p">):</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># If this works, it&#39;s a dict.</span>
<span class="k">return</span> <span class="n">pvalueish</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">pvalueish</span><span class="o">.</span><span class="n">viewvalues</span><span class="p">())</span>
<span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span>
<span class="n">pcolls</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">pvalueish</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pcolls</span><span class="p">,</span> <span class="n">pcolls</span>
<div class="viewcode-block" id="CoGroupByKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.CoGroupByKey.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcolls</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Performs CoGroupByKey on argument pcolls; see class docstring.&quot;&quot;&quot;</span>
<span class="c1"># For associating values in K-V pairs with the PCollections they came from.</span>
<span class="k">def</span> <span class="nf">_pair_tag_with_value</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">),</span> <span class="n">tag</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">))</span>
<span class="c1"># Creates the key, value pairs for the output PCollection. Values are either</span>
<span class="c1"># lists or dicts (per the class docstring), initialized by the result of</span>
<span class="c1"># result_ctor(result_ctor_arg).</span>
<span class="k">def</span> <span class="nf">_merge_tagged_vals_under_key</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">grouped</span><span class="p">),</span> <span class="n">result_ctor</span><span class="p">,</span>
<span class="n">result_ctor_arg</span><span class="p">):</span>
<span class="n">result_value</span> <span class="o">=</span> <span class="n">result_ctor</span><span class="p">(</span><span class="n">result_ctor_arg</span><span class="p">)</span>
<span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">grouped</span><span class="p">:</span>
<span class="n">result_value</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">return</span> <span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">result_value</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># If pcolls is a dict, we turn it into (tag, pcoll) pairs for use in the</span>
<span class="c1"># general-purpose code below. The result value constructor creates dicts</span>
<span class="c1"># whose keys are the tags.</span>
<span class="n">result_ctor_arg</span> <span class="o">=</span> <span class="n">pcolls</span><span class="o">.</span><span class="n">keys</span><span class="p">()</span>
<span class="n">result_ctor</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">tags</span><span class="p">:</span> <span class="nb">dict</span><span class="p">((</span><span class="n">tag</span><span class="p">,</span> <span class="p">[])</span> <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="n">tags</span><span class="p">)</span>
<span class="n">pcolls</span> <span class="o">=</span> <span class="n">pcolls</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span>
<span class="c1"># Otherwise, pcolls is a list/tuple, so we turn it into (index, pcoll)</span>
<span class="c1"># pairs. The result value constructor makes tuples with len(pcolls) slots.</span>
<span class="n">pcolls</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">enumerate</span><span class="p">(</span><span class="n">pcolls</span><span class="p">))</span>
<span class="n">result_ctor_arg</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">pcolls</span><span class="p">)</span>
<span class="n">result_ctor</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">size</span><span class="p">:</span> <span class="nb">tuple</span><span class="p">([]</span> <span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="n">xrange</span><span class="p">(</span><span class="n">size</span><span class="p">))</span>
<span class="c1"># Check input PCollections for PCollection-ness, and that they all belong</span>
<span class="c1"># to the same pipeline.</span>
<span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_check_pcollection</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">:</span>
<span class="k">assert</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span>
<span class="k">return</span> <span class="p">([</span><span class="n">pcoll</span> <span class="o">|</span> <span class="s1">&#39;pair_with_</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">tag</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="n">_pair_tag_with_value</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="p">]</span>
<span class="o">|</span> <span class="n">Flatten</span><span class="p">(</span><span class="n">pipeline</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span>
<span class="o">|</span> <span class="n">GroupByKey</span><span class="p">()</span>
<span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="n">_merge_tagged_vals_under_key</span><span class="p">,</span> <span class="n">result_ctor</span><span class="p">,</span> <span class="n">result_ctor_arg</span><span class="p">))</span></div></div>
<div class="viewcode-block" id="Keys"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Keys">[docs]</a><span class="k">def</span> <span class="nf">Keys</span><span class="p">(</span><span class="n">label</span><span class="o">=</span><span class="s1">&#39;Keys&#39;</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Produces a PCollection of first elements of 2-tuples in a PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">):</span> <span class="n">k</span><span class="p">)</span></div>
<div class="viewcode-block" id="Values"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Values">[docs]</a><span class="k">def</span> <span class="nf">Values</span><span class="p">(</span><span class="n">label</span><span class="o">=</span><span class="s1">&#39;Values&#39;</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Produces a PCollection of second elements of 2-tuples in a PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">):</span> <span class="n">v</span><span class="p">)</span></div>
<div class="viewcode-block" id="KvSwap"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.KvSwap">[docs]</a><span class="k">def</span> <span class="nf">KvSwap</span><span class="p">(</span><span class="n">label</span><span class="o">=</span><span class="s1">&#39;KvSwap&#39;</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Produces a PCollection reversing 2-tuples in a PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">):</span> <span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">k</span><span class="p">))</span></div>
<span class="nd">@ptransform_fn</span>
<span class="k">def</span> <span class="nf">RemoveDuplicates</span><span class="p">(</span><span class="n">pcoll</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Produces a PCollection containing the unique elements of a PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="p">(</span><span class="n">pcoll</span>
<span class="o">|</span> <span class="s1">&#39;ToPairs&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span>
<span class="o">|</span> <span class="s1">&#39;Group&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">CombinePerKey</span><span class="p">(</span><span class="k">lambda</span> <span class="n">vs</span><span class="p">:</span> <span class="kc">None</span><span class="p">)</span>
<span class="o">|</span> <span class="s1">&#39;RemoveDuplicates&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Keys</span><span class="p">())</span>
</pre></div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright .
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
var DOCUMENTATION_OPTIONS = {
URL_ROOT:'../../../',
VERSION:'',
COLLAPSE_INDEX:false,
FILE_SUFFIX:'.html',
HAS_SOURCE: true,
SOURCELINK_SUFFIX: '.txt'
};
</script>
<script type="text/javascript" src="../../../_static/jquery.js"></script>
<script type="text/javascript" src="../../../_static/underscore.js"></script>
<script type="text/javascript" src="../../../_static/doctools.js"></script>
<script type="text/javascript" src="../../../_static/js/theme.js"></script>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.StickyNav.enable();
});
</script>
</body>
</html>