blob: 3de4ad895ada073189466e0018a88de1a86a96e6 [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.pvalue &mdash; Apache Beam documentation</title>
<link rel="stylesheet" href="../../_static/css/theme.css" type="text/css" />
<link rel="index" title="Index"
href="../../genindex.html"/>
<link rel="search" title="Search" href="../../search.html"/>
<link rel="top" title="Apache Beam documentation" href="../../index.html"/>
<link rel="up" title="Module code" href="../index.html"/>
<script src="../../_static/js/modernizr.min.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search">
<a href="../../index.html" class="icon icon-home"> Apache Beam
</a>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.internal.html">apache_beam.internal package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.tools.html">apache_beam.tools package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.utils.html">apache_beam.utils package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../apache_beam.version.html">apache_beam.version module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../index.html">Docs</a> &raquo;</li>
<li><a href="../index.html">Module code</a> &raquo;</li>
<li>apache_beam.pvalue</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.pvalue</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;PValue, PCollection: one node of a dataflow graph.</span>
<span class="sd">A node of a dataflow processing graph is a PValue. Currently, there is only</span>
<span class="sd">one type: PCollection (a potentially very large set of arbitrary values).</span>
<span class="sd">Once created, a PValue belongs to a pipeline and has an associated</span>
<span class="sd">transform (of type PTransform), which describes how the value will be</span>
<span class="sd">produced when the pipeline gets executed.</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">import</span> <span class="nn">collections</span>
<span class="kn">import</span> <span class="nn">itertools</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">hex</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">object</span>
<span class="kn">from</span> <span class="nn">past.builtins</span> <span class="k">import</span> <span class="n">unicode</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">coders</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">typehints</span>
<span class="kn">from</span> <span class="nn">apache_beam.internal</span> <span class="k">import</span> <span class="n">pickler</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="k">import</span> <span class="n">common_urns</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="k">import</span> <span class="n">python_urns</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="k">import</span> <span class="n">beam_runner_api_pb2</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span>
<span class="s1">&#39;PCollection&#39;</span><span class="p">,</span>
<span class="s1">&#39;TaggedOutput&#39;</span><span class="p">,</span>
<span class="s1">&#39;AsSingleton&#39;</span><span class="p">,</span>
<span class="s1">&#39;AsIter&#39;</span><span class="p">,</span>
<span class="s1">&#39;AsList&#39;</span><span class="p">,</span>
<span class="s1">&#39;AsDict&#39;</span><span class="p">,</span>
<span class="s1">&#39;EmptySideInput&#39;</span><span class="p">,</span>
<span class="p">]</span>
<span class="k">class</span> <span class="nc">PValue</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Base class for PCollection.</span>
<span class="sd"> Dataflow users should not construct PValue objects directly in their</span>
<span class="sd"> pipelines.</span>
<span class="sd"> A PValue has the following main characteristics:</span>
<span class="sd"> (1) Belongs to a pipeline. Added during object initialization.</span>
<span class="sd"> (2) Has a transform that can compute the value if executed.</span>
<span class="sd"> (3) Has a value which is meaningful if the transform was executed.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">tag</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">element_type</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">windowing</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes a PValue with all arguments hidden behind keyword arguments.</span>
<span class="sd"> Args:</span>
<span class="sd"> pipeline: Pipeline object for this PValue.</span>
<span class="sd"> tag: Tag of this PValue.</span>
<span class="sd"> element_type: The type of this PValue.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">=</span> <span class="n">pipeline</span>
<span class="bp">self</span><span class="o">.</span><span class="n">tag</span> <span class="o">=</span> <span class="n">tag</span>
<span class="bp">self</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="n">element_type</span>
<span class="c1"># The AppliedPTransform instance for the application of the PTransform</span>
<span class="c1"># generating this PValue. The field gets initialized when a transform</span>
<span class="c1"># gets applied.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">producer</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">windowing</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_windowing</span> <span class="o">=</span> <span class="n">windowing</span>
<span class="k">def</span> <span class="nf">__str__</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">_str_internal</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;&lt;</span><span class="si">%s</span><span class="s1"> at </span><span class="si">%s</span><span class="s1">&gt;&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_str_internal</span><span class="p">(),</span> <span class="nb">hex</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="bp">self</span><span class="p">)))</span>
<span class="k">def</span> <span class="nf">_str_internal</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s2">&quot;</span><span class="si">%s</span><span class="s2">[</span><span class="si">%s</span><span class="s2">.</span><span class="si">%s</span><span class="s2">]&quot;</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</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="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">full_label</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">producer</span> <span class="k">else</span> <span class="kc">None</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">tag</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="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="sd">&quot;&quot;&quot;Applies a transform or callable to a PValue.</span>
<span class="sd"> Args:</span>
<span class="sd"> *args: positional arguments.</span>
<span class="sd"> **kwargs: keyword arguments.</span>
<span class="sd"> The method will insert the pvalue as the next argument following an</span>
<span class="sd"> optional first label and a transform/callable object. It will call the</span>
<span class="sd"> pipeline.apply() method with this modified argument list.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">arglist</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">args</span><span class="p">)</span>
<span class="n">arglist</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="mi">1</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">pipeline</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="o">*</span><span class="n">arglist</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">__or__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">ptransform</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">ptransform</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span>
<div class="viewcode-block" id="PCollection"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.PCollection">[docs]</a><span class="k">class</span> <span class="nc">PCollection</span><span class="p">(</span><span class="n">PValue</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A multiple values (potentially huge) container.</span>
<span class="sd"> Dataflow users should not construct PCollection objects directly in their</span>
<span class="sd"> pipelines.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">PCollection</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">tag</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">tag</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">producer</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">producer</span>
<span class="k">def</span> <span class="nf">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">((</span><span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="p">))</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">windowing</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s1">&#39;_windowing&#39;</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_windowing</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">get_windowing</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">inputs</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_windowing</span>
<span class="k">def</span> <span class="nf">__reduce_ex__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_version</span><span class="p">):</span>
<span class="c1"># Pickling a PCollection is almost always the wrong thing to do, but we</span>
<span class="c1"># can&#39;t prohibit it as it often gets implicitly picked up (e.g. as part</span>
<span class="c1"># of a closure).</span>
<span class="k">return</span> <span class="n">_InvalidUnpickledPCollection</span><span class="p">,</span> <span class="p">()</span>
<div class="viewcode-block" id="PCollection.to_runner_api"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.PCollection.to_runner_api">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">PCollection</span><span class="p">(</span>
<span class="n">unique_name</span><span class="o">=</span><span class="s1">&#39;</span><span class="si">%d%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span>
<span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">full_label</span><span class="p">),</span> <span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">),</span>
<span class="n">coder_id</span><span class="o">=</span><span class="n">pickler</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">element_type</span><span class="p">),</span>
<span class="n">is_bounded</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">IsBounded</span><span class="o">.</span><span class="n">BOUNDED</span><span class="p">,</span>
<span class="n">windowing_strategy_id</span><span class="o">=</span><span class="n">context</span><span class="o">.</span><span class="n">windowing_strategies</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">windowing</span><span class="p">))</span></div>
<div class="viewcode-block" id="PCollection.from_runner_api"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.PCollection.from_runner_api">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># Producer and tag will be filled in later, the key point is that the</span>
<span class="c1"># same object is returned for the same pcollection id.</span>
<span class="k">return</span> <span class="n">PCollection</span><span class="p">(</span>
<span class="kc">None</span><span class="p">,</span>
<span class="n">element_type</span><span class="o">=</span><span class="n">pickler</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">coder_id</span><span class="p">),</span>
<span class="n">windowing</span><span class="o">=</span><span class="n">context</span><span class="o">.</span><span class="n">windowing_strategies</span><span class="o">.</span><span class="n">get_by_id</span><span class="p">(</span>
<span class="n">proto</span><span class="o">.</span><span class="n">windowing_strategy_id</span><span class="p">))</span></div></div>
<span class="k">class</span> <span class="nc">_InvalidUnpickledPCollection</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">class</span> <span class="nc">PBegin</span><span class="p">(</span><span class="n">PValue</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A pipeline begin marker used as input to create/read transforms.</span>
<span class="sd"> The class is used internally to represent inputs to Create and Read</span>
<span class="sd"> transforms. This allows us to have transforms that uniformly take PValue(s)</span>
<span class="sd"> as inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span>
<span class="k">class</span> <span class="nc">PDone</span><span class="p">(</span><span class="n">PValue</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;PDone is the output of a transform that has a trivial result such as Write.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span>
<span class="k">class</span> <span class="nc">DoOutputsTuple</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;An object grouping the multiple outputs of a ParDo or FlatMap transform.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">transform</span><span class="p">,</span> <span class="n">tags</span><span class="p">,</span> <span class="n">main_tag</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">pipeline</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_tags</span> <span class="o">=</span> <span class="n">tags</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_main_tag</span> <span class="o">=</span> <span class="n">main_tag</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform</span> <span class="o">=</span> <span class="n">transform</span>
<span class="c1"># The ApplyPTransform instance for the application of the multi FlatMap</span>
<span class="c1"># generating this value. The field gets initialized when a transform</span>
<span class="c1"># gets applied.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">producer</span> <span class="o">=</span> <span class="kc">None</span>
<span class="c1"># Dictionary of PCollections already associated with tags.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pcolls</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">def</span> <span class="nf">__str__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;&lt;</span><span class="si">%s</span><span class="s1">&gt;&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_str_internal</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;&lt;</span><span class="si">%s</span><span class="s1"> at </span><span class="si">%s</span><span class="s1">&gt;&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_str_internal</span><span class="p">(),</span> <span class="nb">hex</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="bp">self</span><span class="p">)))</span>
<span class="k">def</span> <span class="nf">_str_internal</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;</span><span class="si">%s</span><span class="s1"> main_tag=</span><span class="si">%s</span><span class="s1"> tags=</span><span class="si">%s</span><span class="s1"> transform=</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span>
<span class="bp">self</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="bp">self</span><span class="o">.</span><span class="n">_main_tag</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_tags</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_transform</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">__iter__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Iterates over tags returning for each call a (tag, pvalue) pair.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_main_tag</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">yield</span> <span class="bp">self</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_main_tag</span><span class="p">]</span>
<span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_tags</span><span class="p">:</span>
<span class="k">yield</span> <span class="bp">self</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">__getattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="c1"># Special methods which may be accessed before the object is</span>
<span class="c1"># fully constructed (e.g. in unpickling).</span>
<span class="k">if</span> <span class="n">tag</span><span class="p">[:</span><span class="mi">2</span><span class="p">]</span> <span class="o">==</span> <span class="n">tag</span><span class="p">[</span><span class="o">-</span><span class="mi">2</span><span class="p">:]</span> <span class="o">==</span> <span class="s1">&#39;__&#39;</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">object</span><span class="o">.</span><span class="fm">__getattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">__getitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="c1"># Accept int tags so that we can look at Partition tags with the</span>
<span class="c1"># same ints that we used in the partition function.</span>
<span class="c1"># TODO(gildea): Consider requiring string-based tags everywhere.</span>
<span class="c1"># This will require a partition function that does not return ints.</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="n">tag</span> <span class="o">=</span> <span class="nb">str</span><span class="p">(</span><span class="n">tag</span><span class="p">)</span>
<span class="k">if</span> <span class="n">tag</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">_main_tag</span><span class="p">:</span>
<span class="n">tag</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">_tags</span> <span class="ow">and</span> <span class="n">tag</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_tags</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Tag &#39;</span><span class="si">%s</span><span class="s2">&#39; is neither the main tag &#39;</span><span class="si">%s</span><span class="s2">&#39; &quot;</span>
<span class="s2">&quot;nor any of the tags </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="p">(</span>
<span class="n">tag</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_main_tag</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_tags</span><span class="p">))</span>
<span class="c1"># Check if we accessed this tag before.</span>
<span class="k">if</span> <span class="n">tag</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcolls</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcolls</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span>
<span class="k">if</span> <span class="n">tag</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform</span><span class="o">.</span><span class="n">output_tags</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">tag</span><span class="p">)</span>
<span class="n">pcoll</span> <span class="o">=</span> <span class="n">PCollection</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">,</span> <span class="n">tag</span><span class="o">=</span><span class="n">tag</span><span class="p">)</span>
<span class="c1"># Transfer the producer from the DoOutputsTuple to the resulting</span>
<span class="c1"># PCollection.</span>
<span class="n">pcoll</span><span class="o">.</span><span class="n">producer</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">parts</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="c1"># Add this as an output to both the inner ParDo and the outer _MultiParDo</span>
<span class="c1"># PTransforms.</span>
<span class="k">if</span> <span class="n">tag</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">parts</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">outputs</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">parts</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">add_output</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">add_output</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Main output is output of inner ParDo.</span>
<span class="n">pcoll</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">parts</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">outputs</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">_pcolls</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span> <span class="o">=</span> <span class="n">pcoll</span>
<span class="k">return</span> <span class="n">pcoll</span>
<div class="viewcode-block" id="TaggedOutput"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.TaggedOutput">[docs]</a><span class="k">class</span> <span class="nc">TaggedOutput</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;An object representing a tagged value.</span>
<span class="sd"> ParDo, Map, and FlatMap transforms can emit values on multiple outputs which</span>
<span class="sd"> are distinguished by string tags. The DoFn will return plain values</span>
<span class="sd"> if it wants to emit on the main output and TaggedOutput objects</span>
<span class="sd"> if it wants to emit a value on a specific tagged output.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="p">(</span><span class="nb">str</span><span class="p">,</span> <span class="n">unicode</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;Attempting to create a TaggedOutput with non-string tag </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">tag</span><span class="p">,))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">tag</span> <span class="o">=</span> <span class="n">tag</span>
<span class="bp">self</span><span class="o">.</span><span class="n">value</span> <span class="o">=</span> <span class="n">value</span></div>
<span class="k">class</span> <span class="nc">AsSideInput</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Marker specifying that a PCollection will be used as a side input.</span>
<span class="sd"> When a PCollection is supplied as a side input to a PTransform, it is</span>
<span class="sd"> necessary to indicate how the PCollection should be made available</span>
<span class="sd"> as a PTransform side argument (e.g. in the form of an iterable, mapping,</span>
<span class="sd"> or single value). This class is the superclass of all the various</span>
<span class="sd"> options, and should not be instantiated directly. (See instead AsSingleton,</span>
<span class="sd"> AsIter, etc.)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="k">import</span> <span class="n">sideinputs</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pvalue</span> <span class="o">=</span> <span class="n">pcoll</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window_mapping_fn</span> <span class="o">=</span> <span class="n">sideinputs</span><span class="o">.</span><span class="n">default_window_mapping_fn</span><span class="p">(</span>
<span class="n">pcoll</span><span class="o">.</span><span class="n">windowing</span><span class="o">.</span><span class="n">windowfn</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_view_options</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Internal options corresponding to specific view.</span>
<span class="sd"> Intended for internal use by runner implementations.</span>
<span class="sd"> Returns:</span>
<span class="sd"> Tuple of options for the given view.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="p">{</span><span class="s1">&#39;window_mapping_fn&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_window_mapping_fn</span><span class="p">}</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">element_type</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span>
<span class="c1"># TODO(robertwb): Get rid of _from_runtime_iterable and _view_options</span>
<span class="c1"># in favor of _side_input_data().</span>
<span class="k">def</span> <span class="nf">_side_input_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">view_options</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_view_options</span><span class="p">()</span>
<span class="n">from_runtime_iterable</span> <span class="o">=</span> <span class="nb">type</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="n">_from_runtime_iterable</span>
<span class="k">return</span> <span class="n">SideInputData</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">ITERABLE</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window_mapping_fn</span><span class="p">,</span>
<span class="k">lambda</span> <span class="n">iterable</span><span class="p">:</span> <span class="n">from_runtime_iterable</span><span class="p">(</span><span class="n">iterable</span><span class="p">,</span> <span class="n">view_options</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">_input_element_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">coders</span><span class="o">.</span><span class="n">WindowedValueCoder</span><span class="p">(</span>
<span class="n">coders</span><span class="o">.</span><span class="n">registry</span><span class="o">.</span><span class="n">get_coder</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">element_type</span><span class="p">),</span>
<span class="n">window_coder</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">windowing</span><span class="o">.</span><span class="n">windowfn</span><span class="o">.</span><span class="n">get_window_coder</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_side_input_data</span><span class="p">()</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">(</span><span class="n">context</span><span class="p">)</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_UnpickledSideInput</span><span class="p">(</span>
<span class="n">SideInputData</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="p">,</span> <span class="n">context</span><span class="p">))</span>
<span class="k">class</span> <span class="nc">_UnpickledSideInput</span><span class="p">(</span><span class="n">AsSideInput</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">side_input_data</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_data</span> <span class="o">=</span> <span class="n">side_input_data</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window_mapping_fn</span> <span class="o">=</span> <span class="n">side_input_data</span><span class="o">.</span><span class="n">window_mapping_fn</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_from_runtime_iterable</span><span class="p">(</span><span class="n">it</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="k">return</span> <span class="n">options</span><span class="p">[</span><span class="s1">&#39;data&#39;</span><span class="p">]</span><span class="o">.</span><span class="n">view_fn</span><span class="p">(</span><span class="n">it</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_view_options</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;data&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">,</span>
<span class="c1"># For non-fn-api runners.</span>
<span class="s1">&#39;window_mapping_fn&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="o">.</span><span class="n">window_mapping_fn</span><span class="p">,</span>
<span class="p">}</span>
<span class="k">def</span> <span class="nf">_side_input_data</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">_data</span>
<span class="k">class</span> <span class="nc">SideInputData</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;All of the data about a side input except for the bound PCollection.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">access_pattern</span><span class="p">,</span> <span class="n">window_mapping_fn</span><span class="p">,</span> <span class="n">view_fn</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">access_pattern</span> <span class="o">=</span> <span class="n">access_pattern</span>
<span class="bp">self</span><span class="o">.</span><span class="n">window_mapping_fn</span> <span class="o">=</span> <span class="n">window_mapping_fn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">view_fn</span> <span class="o">=</span> <span class="n">view_fn</span>
<span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">SideInput</span><span class="p">(</span>
<span class="n">access_pattern</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">FunctionSpec</span><span class="p">(</span>
<span class="n">urn</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">access_pattern</span><span class="p">),</span>
<span class="n">view_fn</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">SdkFunctionSpec</span><span class="p">(</span>
<span class="n">environment_id</span><span class="o">=</span><span class="n">context</span><span class="o">.</span><span class="n">default_environment_id</span><span class="p">(),</span>
<span class="n">spec</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">FunctionSpec</span><span class="p">(</span>
<span class="n">urn</span><span class="o">=</span><span class="n">python_urns</span><span class="o">.</span><span class="n">PICKLED_VIEWFN</span><span class="p">,</span>
<span class="n">payload</span><span class="o">=</span><span class="n">pickler</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">view_fn</span><span class="p">))),</span>
<span class="n">window_mapping_fn</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">SdkFunctionSpec</span><span class="p">(</span>
<span class="n">environment_id</span><span class="o">=</span><span class="n">context</span><span class="o">.</span><span class="n">default_environment_id</span><span class="p">(),</span>
<span class="n">spec</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">FunctionSpec</span><span class="p">(</span>
<span class="n">urn</span><span class="o">=</span><span class="n">python_urns</span><span class="o">.</span><span class="n">PICKLED_WINDOW_MAPPING_FN</span><span class="p">,</span>
<span class="n">payload</span><span class="o">=</span><span class="n">pickler</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">window_mapping_fn</span><span class="p">))))</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">assert</span> <span class="n">proto</span><span class="o">.</span><span class="n">view_fn</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span> <span class="n">python_urns</span><span class="o">.</span><span class="n">PICKLED_VIEWFN</span>
<span class="k">assert</span> <span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">window_mapping_fn</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span>
<span class="n">python_urns</span><span class="o">.</span><span class="n">PICKLED_WINDOW_MAPPING_FN</span><span class="p">)</span>
<span class="k">return</span> <span class="n">SideInputData</span><span class="p">(</span>
<span class="n">proto</span><span class="o">.</span><span class="n">access_pattern</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">pickler</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">window_mapping_fn</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">payload</span><span class="p">),</span>
<span class="n">pickler</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">view_fn</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">payload</span><span class="p">))</span>
<div class="viewcode-block" id="AsSingleton"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.AsSingleton">[docs]</a><span class="k">class</span> <span class="nc">AsSingleton</span><span class="p">(</span><span class="n">AsSideInput</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Marker specifying that an entire PCollection is to be used as a side input.</span>
<span class="sd"> When a PCollection is supplied as a side input to a PTransform, it is</span>
<span class="sd"> necessary to indicate whether the entire PCollection should be made available</span>
<span class="sd"> as a PTransform side argument (in the form of an iterable), or whether just</span>
<span class="sd"> one value should be pulled from the PCollection and supplied as the side</span>
<span class="sd"> argument (as an ordinary value).</span>
<span class="sd"> Wrapping a PCollection side input argument to a PTransform in this container</span>
<span class="sd"> (e.g., data.apply(&#39;label&#39;, MyPTransform(), AsSingleton(my_side_input) )</span>
<span class="sd"> selects the latter behavor.</span>
<span class="sd"> The input PCollection must contain exactly one value per window, unless a</span>
<span class="sd"> default is given, in which case it may be empty.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">_NO_DEFAULT</span> <span class="o">=</span> <span class="nb">object</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">,</span> <span class="n">default_value</span><span class="o">=</span><span class="n">_NO_DEFAULT</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">AsSingleton</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="n">pcoll</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">default_value</span> <span class="o">=</span> <span class="n">default_value</span>
<span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;AsSingleton(</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">pvalue</span>
<span class="k">def</span> <span class="nf">_view_options</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">base</span> <span class="o">=</span> <span class="nb">super</span><span class="p">(</span><span class="n">AsSingleton</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="n">_view_options</span><span class="p">()</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">default_value</span> <span class="o">!=</span> <span class="n">AsSingleton</span><span class="o">.</span><span class="n">_NO_DEFAULT</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">dict</span><span class="p">(</span><span class="n">base</span><span class="p">,</span> <span class="n">default</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">default_value</span><span class="p">)</span>
<span class="k">return</span> <span class="n">base</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_from_runtime_iterable</span><span class="p">(</span><span class="n">it</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="n">head</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">it</span><span class="p">,</span> <span class="mi">2</span><span class="p">))</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">head</span><span class="p">:</span>
<span class="k">return</span> <span class="n">options</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;default&#39;</span><span class="p">,</span> <span class="n">EmptySideInput</span><span class="p">())</span>
<span class="k">elif</span> <span class="nb">len</span><span class="p">(</span><span class="n">head</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">return</span> <span class="n">head</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;PCollection of size </span><span class="si">%d</span><span class="s1"> with more than one element accessed as a &#39;</span>
<span class="s1">&#39;singleton view. First two elements encountered are &quot;</span><span class="si">%s</span><span class="s1">&quot;, &quot;</span><span class="si">%s</span><span class="s1">&quot;.&#39;</span> <span class="o">%</span> <span class="p">(</span>
<span class="nb">len</span><span class="p">(</span><span class="n">head</span><span class="p">),</span> <span class="nb">str</span><span class="p">(</span><span class="n">head</span><span class="p">[</span><span class="mi">0</span><span class="p">]),</span> <span class="nb">str</span><span class="p">(</span><span class="n">head</span><span class="p">[</span><span class="mi">1</span><span class="p">])))</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">element_type</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">pvalue</span><span class="o">.</span><span class="n">element_type</span></div>
<div class="viewcode-block" id="AsIter"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.AsIter">[docs]</a><span class="k">class</span> <span class="nc">AsIter</span><span class="p">(</span><span class="n">AsSideInput</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Marker specifying that an entire PCollection is to be used as a side input.</span>
<span class="sd"> When a PCollection is supplied as a side input to a PTransform, it is</span>
<span class="sd"> necessary to indicate whether the entire PCollection should be made available</span>
<span class="sd"> as a PTransform side argument (in the form of an iterable), or whether just</span>
<span class="sd"> one value should be pulled from the PCollection and supplied as the side</span>
<span class="sd"> argument (as an ordinary value).</span>
<span class="sd"> Wrapping a PCollection side input argument to a PTransform in this container</span>
<span class="sd"> (e.g., data.apply(&#39;label&#39;, MyPTransform(), AsIter(my_side_input) ) selects the</span>
<span class="sd"> former behavor.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;AsIter(</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">pvalue</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_from_runtime_iterable</span><span class="p">(</span><span class="n">it</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="k">return</span> <span class="n">it</span>
<span class="k">def</span> <span class="nf">_side_input_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">SideInputData</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">ITERABLE</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window_mapping_fn</span><span class="p">,</span>
<span class="k">lambda</span> <span class="n">iterable</span><span class="p">:</span> <span class="n">iterable</span><span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">element_type</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">element_type</span><span class="p">]</span></div>
<div class="viewcode-block" id="AsList"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.AsList">[docs]</a><span class="k">class</span> <span class="nc">AsList</span><span class="p">(</span><span class="n">AsSideInput</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Marker specifying that an entire PCollection is to be used as a side input.</span>
<span class="sd"> Intended for use in side-argument specification---the same places where</span>
<span class="sd"> AsSingleton and AsIter are used, but forces materialization of this</span>
<span class="sd"> PCollection as a list.</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll: Input pcollection.</span>
<span class="sd"> Returns:</span>
<span class="sd"> An AsList-wrapper around a PCollection whose one element is a list</span>
<span class="sd"> containing all elements in pcoll.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_from_runtime_iterable</span><span class="p">(</span><span class="n">it</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="n">it</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_side_input_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">SideInputData</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">ITERABLE</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window_mapping_fn</span><span class="p">,</span>
<span class="nb">list</span><span class="p">)</span></div>
<div class="viewcode-block" id="AsDict"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.AsDict">[docs]</a><span class="k">class</span> <span class="nc">AsDict</span><span class="p">(</span><span class="n">AsSideInput</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Marker specifying a PCollection to be used as an indexable side input.</span>
<span class="sd"> Intended for use in side-argument specification---the same places where</span>
<span class="sd"> AsSingleton and AsIter are used, but returns an interface that allows</span>
<span class="sd"> key lookup.</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll: Input pcollection. All elements should be key-value pairs (i.e.</span>
<span class="sd"> 2-tuples) with unique keys.</span>
<span class="sd"> Returns:</span>
<span class="sd"> An AsDict-wrapper around a PCollection whose one element is a dict with</span>
<span class="sd"> entries for uniquely-keyed pairs in pcoll.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_from_runtime_iterable</span><span class="p">(</span><span class="n">it</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">dict</span><span class="p">(</span><span class="n">it</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_side_input_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">SideInputData</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">ITERABLE</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window_mapping_fn</span><span class="p">,</span>
<span class="nb">dict</span><span class="p">)</span></div>
<span class="k">class</span> <span class="nc">AsMultiMap</span><span class="p">(</span><span class="n">AsSideInput</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Marker specifying a PCollection to be used as an indexable side input.</span>
<span class="sd"> Similar to AsDict, but multiple values may be associated per key, and</span>
<span class="sd"> the keys are fetched lazily rather than all having to fit in memory.</span>
<span class="sd"> Intended for use in side-argument specification---the same places where</span>
<span class="sd"> AsSingleton and AsIter are used, but returns an interface that allows</span>
<span class="sd"> key lookup.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_from_runtime_iterable</span><span class="p">(</span><span class="n">it</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># Legacy implementation.</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">)</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">it</span><span class="p">:</span>
<span class="n">result</span><span class="p">[</span><span class="n">k</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">v</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span>
<span class="k">def</span> <span class="nf">_side_input_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">SideInputData</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">MULTIMAP</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window_mapping_fn</span><span class="p">,</span>
<span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">)</span>
<div class="viewcode-block" id="EmptySideInput"><a class="viewcode-back" href="../../apache_beam.pvalue.html#apache_beam.pvalue.EmptySideInput">[docs]</a><span class="k">class</span> <span class="nc">EmptySideInput</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Value indicating when a singleton side input was empty.</span>
<span class="sd"> If a PCollection was furnished as a singleton side input to a PTransform, and</span>
<span class="sd"> that PCollection was empty, then this value is supplied to the DoFn in the</span>
<span class="sd"> place where a value from a non-empty PCollection would have gone. This alerts</span>
<span class="sd"> the DoFn that the side input PCollection was empty. Users may want to check</span>
<span class="sd"> whether side input values are EmptySideInput, but they will very likely never</span>
<span class="sd"> want to create new instances of this class themselves.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
</pre></div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright .
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
var DOCUMENTATION_OPTIONS = {
URL_ROOT:'../../',
VERSION:'',
COLLAPSE_INDEX:false,
FILE_SUFFIX:'.html',
HAS_SOURCE: true,
SOURCELINK_SUFFIX: '.txt'
};
</script>
<script type="text/javascript" src="../../_static/jquery.js"></script>
<script type="text/javascript" src="../../_static/underscore.js"></script>
<script type="text/javascript" src="../../_static/doctools.js"></script>
<script type="text/javascript" src="../../_static/js/theme.js"></script>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.StickyNav.enable();
});
</script>
</body>
</html>