blob: 6a036ab4ff4a47e8c2dff3ac993174dc09703cae [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.core &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.core</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.core</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;Core PTransform subclasses, such as FlatMap, GroupByKey, and Map.&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span> <span class="nn">concurrent.futures</span>
<span class="kn">import</span> <span class="nn">copy</span>
<span class="kn">import</span> <span class="nn">inspect</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">import</span> <span class="nn">random</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">import</span> <span class="nn">traceback</span>
<span class="kn">import</span> <span class="nn">types</span>
<span class="kn">import</span> <span class="nn">typing</span>
<span class="kn">from</span> <span class="nn">itertools</span> <span class="kn">import</span> <span class="n">dropwhile</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">coders</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">pvalue</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.coders</span> <span class="kn">import</span> <span class="n">typecoders</span>
<span class="kn">from</span> <span class="nn">apache_beam.internal</span> <span class="kn">import</span> <span class="n">pickler</span>
<span class="kn">from</span> <span class="nn">apache_beam.internal</span> <span class="kn">import</span> <span class="n">util</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="kn">import</span> <span class="n">TypeOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="kn">import</span> <span class="n">common_urns</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="kn">import</span> <span class="n">python_urns</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">beam_runner_api_pb2</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">userstate</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.transforms.display</span> <span class="kn">import</span> <span class="n">HasDisplayData</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="kn">import</span> <span class="n">PTransform</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="kn">import</span> <span class="n">PTransformWithSideInputs</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.sideinputs</span> <span class="kn">import</span> <span class="n">SIDE_INPUT_PREFIX</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.sideinputs</span> <span class="kn">import</span> <span class="n">get_sideinput_index</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.userstate</span> <span class="kn">import</span> <span class="n">StateSpec</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.userstate</span> <span class="kn">import</span> <span class="n">TimerSpec</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">GlobalWindows</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">SlidingWindows</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">TimestampCombiner</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">TimestampedValue</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">WindowedValue</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">WindowFn</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints</span> <span class="kn">import</span> <span class="n">row_type</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints</span> <span class="kn">import</span> <span class="n">trivial_inference</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.batch</span> <span class="kn">import</span> <span class="n">BatchConverter</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.decorators</span> <span class="kn">import</span> <span class="n">TypeCheckError</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.decorators</span> <span class="kn">import</span> <span class="n">WithTypeHints</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.decorators</span> <span class="kn">import</span> <span class="n">get_signature</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.decorators</span> <span class="kn">import</span> <span class="n">get_type_hints</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.decorators</span> <span class="kn">import</span> <span class="n">with_input_types</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.decorators</span> <span class="kn">import</span> <span class="n">with_output_types</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.trivial_inference</span> <span class="kn">import</span> <span class="n">element_type</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.typehints</span> <span class="kn">import</span> <span class="n">TypeConstraint</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.typehints</span> <span class="kn">import</span> <span class="n">is_consistent_with</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.typehints</span> <span class="kn">import</span> <span class="n">visit_inner_types</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="kn">import</span> <span class="n">urns</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="k">if</span> <span class="n">typing</span><span class="o">.</span><span class="n">TYPE_CHECKING</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">google.protobuf</span> <span class="kn">import</span> <span class="n">message</span> <span class="c1"># pylint: disable=ungrouped-imports</span>
<span class="kn">from</span> <span class="nn">apache_beam.io</span> <span class="kn">import</span> <span class="n">iobase</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="kn">import</span> <span class="n">Pipeline</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.pipeline_context</span> <span class="kn">import</span> <span class="n">PipelineContext</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">create_source</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">AccumulationMode</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">DefaultTrigger</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">TriggerFn</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span>
<span class="s1">&#39;DoFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;CombineFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;PartitionFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;ParDo&#39;</span><span class="p">,</span>
<span class="s1">&#39;FlatMap&#39;</span><span class="p">,</span>
<span class="s1">&#39;FlatMapTuple&#39;</span><span class="p">,</span>
<span class="s1">&#39;Map&#39;</span><span class="p">,</span>
<span class="s1">&#39;MapTuple&#39;</span><span class="p">,</span>
<span class="s1">&#39;Filter&#39;</span><span class="p">,</span>
<span class="s1">&#39;CombineGlobally&#39;</span><span class="p">,</span>
<span class="s1">&#39;CombinePerKey&#39;</span><span class="p">,</span>
<span class="s1">&#39;CombineValues&#39;</span><span class="p">,</span>
<span class="s1">&#39;GroupBy&#39;</span><span class="p">,</span>
<span class="s1">&#39;GroupByKey&#39;</span><span class="p">,</span>
<span class="s1">&#39;Select&#39;</span><span class="p">,</span>
<span class="s1">&#39;Partition&#39;</span><span class="p">,</span>
<span class="s1">&#39;Windowing&#39;</span><span class="p">,</span>
<span class="s1">&#39;WindowInto&#39;</span><span class="p">,</span>
<span class="s1">&#39;Flatten&#39;</span><span class="p">,</span>
<span class="s1">&#39;Create&#39;</span><span class="p">,</span>
<span class="s1">&#39;Impulse&#39;</span><span class="p">,</span>
<span class="s1">&#39;RestrictionProvider&#39;</span><span class="p">,</span>
<span class="s1">&#39;WatermarkEstimatorProvider&#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">typing</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">typing</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">typing</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">_LOGGER</span> <span class="o">=</span> <span class="n">logging</span><span class="o">.</span><span class="n">getLogger</span><span class="p">(</span><span class="vm">__name__</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">DoFnContext</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A context available to all methods of DoFn instance.&quot;&quot;&quot;</span>
<span class="k">pass</span>
<span class="k">class</span> <span class="nc">DoFnProcessContext</span><span class="p">(</span><span class="n">DoFnContext</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A processing context passed to DoFn process() during execution.</span>
<span class="sd"> Experimental; no backwards-compatibility guarantees.</span>
<span class="sd"> Most importantly, a DoFn.process method will access context.element</span>
<span class="sd"> to get the element it is supposed to process.</span>
<span class="sd"> Attributes:</span>
<span class="sd"> label: label of the ParDo whose element is being processed.</span>
<span class="sd"> element: element being processed</span>
<span class="sd"> (in process method only; always None in start_bundle and finish_bundle)</span>
<span class="sd"> timestamp: timestamp of the element</span>
<span class="sd"> (in process method only; always None in start_bundle and finish_bundle)</span>
<span class="sd"> windows: windows of the element</span>
<span class="sd"> (in process method only; always None in start_bundle and finish_bundle)</span>
<span class="sd"> state: a DoFnState object, which holds the runner&#39;s internal state</span>
<span class="sd"> for this element.</span>
<span class="sd"> Not used by the pipeline code.</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">label</span><span class="p">,</span> <span class="n">element</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">state</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Initialize a processing context object with an element and state.</span>
<span class="sd"> The element represents one value from a PCollection that will be accessed</span>
<span class="sd"> by a DoFn object during pipeline execution, and state is an arbitrary object</span>
<span class="sd"> where counters and other pipeline state information can be passed in.</span>
<span class="sd"> DoFnProcessContext objects are also used as inputs to PartitionFn instances.</span>
<span class="sd"> Args:</span>
<span class="sd"> label: label of the PCollection whose element is being processed.</span>
<span class="sd"> element: element of a PCollection being processed using this context.</span>
<span class="sd"> state: a DoFnState object with state to be passed in to the DoFn object.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">=</span> <span class="n">label</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state</span> <span class="o">=</span> <span class="n">state</span>
<span class="k">if</span> <span class="n">element</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">set_element</span><span class="p">(</span><span class="n">element</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">set_element</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">windowed_value</span><span class="p">):</span>
<span class="k">if</span> <span class="n">windowed_value</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="c1"># Not currently processing an element.</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s1">&#39;element&#39;</span><span class="p">):</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">element</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">timestamp</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">windows</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">element</span> <span class="o">=</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">value</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp</span> <span class="o">=</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">timestamp</span>
<span class="bp">self</span><span class="o">.</span><span class="n">windows</span> <span class="o">=</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">windows</span>
<span class="k">class</span> <span class="nc">ProcessContinuation</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;An object that may be produced as the last element of a process method</span>
<span class="sd"> invocation.</span>
<span class="sd"> Experimental; no backwards-compatibility guarantees.</span>
<span class="sd"> If produced, indicates that there is more work to be done for the current</span>
<span class="sd"> input element.</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">resume_delay</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Initializes a ProcessContinuation object.</span>
<span class="sd"> Args:</span>
<span class="sd"> resume_delay: indicates the minimum time, in seconds, that should elapse</span>
<span class="sd"> before re-invoking process() method for resuming the invocation of the</span>
<span class="sd"> current element.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">resume_delay</span> <span class="o">=</span> <span class="n">resume_delay</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">resume</span><span class="p">(</span><span class="n">resume_delay</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A convenient method that produces a ``ProcessContinuation``.</span>
<span class="sd"> Args:</span>
<span class="sd"> resume_delay: delay after which processing current element should be</span>
<span class="sd"> resumed.</span>
<span class="sd"> Returns: a ``ProcessContinuation`` for signalling the runner that current</span>
<span class="sd"> input element has not been fully processed and should be resumed later.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">ProcessContinuation</span><span class="p">(</span><span class="n">resume_delay</span><span class="o">=</span><span class="n">resume_delay</span><span class="p">)</span>
<div class="viewcode-block" id="RestrictionProvider"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.RestrictionProvider">[docs]</a><span class="k">class</span> <span class="nc">RestrictionProvider</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Provides methods for generating and manipulating restrictions.</span>
<span class="sd"> This class should be implemented to support Splittable ``DoFn`` in Python</span>
<span class="sd"> SDK. See https://s.apache.org/splittable-do-fn for more details about</span>
<span class="sd"> Splittable ``DoFn``.</span>
<span class="sd"> To denote a ``DoFn`` class to be Splittable ``DoFn``, ``DoFn.process()``</span>
<span class="sd"> method of that class should have exactly one parameter whose default value is</span>
<span class="sd"> an instance of ``RestrictionParam``. This ``RestrictionParam`` can either be</span>
<span class="sd"> constructed with an explicit ``RestrictionProvider``, or, if no</span>
<span class="sd"> ``RestrictionProvider`` is provided, the ``DoFn`` itself must be a</span>
<span class="sd"> ``RestrictionProvider``.</span>
<span class="sd"> The provided ``RestrictionProvider`` instance must provide suitable overrides</span>
<span class="sd"> for the following methods:</span>
<span class="sd"> * create_tracker()</span>
<span class="sd"> * initial_restriction()</span>
<span class="sd"> * restriction_size()</span>
<span class="sd"> Optionally, ``RestrictionProvider`` may override default implementations of</span>
<span class="sd"> following methods:</span>
<span class="sd"> * restriction_coder()</span>
<span class="sd"> * split()</span>
<span class="sd"> * split_and_size()</span>
<span class="sd"> * truncate()</span>
<span class="sd"> ** Pausing and resuming processing of an element **</span>
<span class="sd"> As the last element produced by the iterator returned by the</span>
<span class="sd"> ``DoFn.process()`` method, a Splittable ``DoFn`` may return an object of type</span>
<span class="sd"> ``ProcessContinuation``.</span>
<span class="sd"> If restriction_tracker.defer_remander is called in the ```DoFn.process()``, it</span>
<span class="sd"> means that runner should later re-invoke ``DoFn.process()`` method to resume</span>
<span class="sd"> processing the current element and the manner in which the re-invocation</span>
<span class="sd"> should be performed.</span>
<span class="sd"> ** Updating output watermark **</span>
<span class="sd"> ``DoFn.process()`` method of Splittable ``DoFn``s could contain a parameter</span>
<span class="sd"> with default value ``DoFn.WatermarkReporterParam``. If specified this asks the</span>
<span class="sd"> runner to provide a function that can be used to give the runner a</span>
<span class="sd"> (best-effort) lower bound about the timestamps of future output associated</span>
<span class="sd"> with the current element processed by the ``DoFn``. If the ``DoFn`` has</span>
<span class="sd"> multiple outputs, the watermark applies to all of them. Provided function must</span>
<span class="sd"> be invoked with a single parameter of type ``Timestamp`` or as an integer that</span>
<span class="sd"> gives the watermark in number of seconds.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="RestrictionProvider.create_tracker"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.RestrictionProvider.create_tracker">[docs]</a> <span class="k">def</span> <span class="nf">create_tracker</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span>
<span class="c1"># type: (...) -&gt; iobase.RestrictionTracker</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Produces a new ``RestrictionTracker`` for the given restriction.</span>
<span class="sd"> This API is required to be implemented.</span>
<span class="sd"> Args:</span>
<span class="sd"> restriction: an object that defines a restriction as identified by a</span>
<span class="sd"> Splittable ``DoFn`` that utilizes the current ``RestrictionProvider``.</span>
<span class="sd"> For example, a tuple that gives a range of positions for a Splittable</span>
<span class="sd"> ``DoFn`` that reads files based on byte positions.</span>
<span class="sd"> Returns: an object of type ``RestrictionTracker``.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="RestrictionProvider.initial_restriction"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.RestrictionProvider.initial_restriction">[docs]</a> <span class="k">def</span> <span class="nf">initial_restriction</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="w"> </span><span class="sd">&quot;&quot;&quot;Produces an initial restriction for the given element.</span>
<span class="sd"> This API is required to be implemented.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="RestrictionProvider.split"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.RestrictionProvider.split">[docs]</a> <span class="k">def</span> <span class="nf">split</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Splits the given element and restriction initially.</span>
<span class="sd"> This method enables runners to perform bulk splitting initially allowing for</span>
<span class="sd"> a rapid increase in parallelism. Note that initial split is a different</span>
<span class="sd"> concept from the split during element processing time. Please refer to</span>
<span class="sd"> ``iobase.RestrictionTracker.try_split`` for details about splitting when the</span>
<span class="sd"> current element and restriction are actively being processed.</span>
<span class="sd"> Returns an iterator of restrictions. The total set of elements produced by</span>
<span class="sd"> reading input element for each of the returned restrictions should be the</span>
<span class="sd"> same as the total set of elements produced by reading the input element for</span>
<span class="sd"> the input restriction.</span>
<span class="sd"> This API is optional if ``split_and_size`` has been implemented.</span>
<span class="sd"> If this method is not override, there is no initial splitting happening on</span>
<span class="sd"> each restriction.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">yield</span> <span class="n">restriction</span></div>
<div class="viewcode-block" id="RestrictionProvider.restriction_coder"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.RestrictionProvider.restriction_coder">[docs]</a> <span class="k">def</span> <span class="nf">restriction_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns a ``Coder`` for restrictions.</span>
<span class="sd"> Returned``Coder`` will be used for the restrictions produced by the current</span>
<span class="sd"> ``RestrictionProvider``.</span>
<span class="sd"> Returns:</span>
<span class="sd"> an object of type ``Coder``.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</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="nb">object</span><span class="p">)</span></div>
<div class="viewcode-block" id="RestrictionProvider.restriction_size"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.RestrictionProvider.restriction_size">[docs]</a> <span class="k">def</span> <span class="nf">restriction_size</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns the size of a restriction with respect to the given element.</span>
<span class="sd"> By default, asks a newly-created restriction tracker for the default size</span>
<span class="sd"> of the restriction.</span>
<span class="sd"> The return value must be non-negative.</span>
<span class="sd"> Must be thread safe. Will be invoked concurrently during bundle processing</span>
<span class="sd"> due to runner initiated splitting and progress estimation.</span>
<span class="sd"> This API is required to be implemented.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="RestrictionProvider.split_and_size"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.RestrictionProvider.split_and_size">[docs]</a> <span class="k">def</span> <span class="nf">split_and_size</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Like split, but also does sizing, returning (restriction, size) pairs.</span>
<span class="sd"> For each pair, size must be non-negative.</span>
<span class="sd"> This API is optional if ``split`` and ``restriction_size`` have been</span>
<span class="sd"> implemented.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">for</span> <span class="n">part</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">part</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">restriction_size</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">part</span><span class="p">)</span></div>
<div class="viewcode-block" id="RestrictionProvider.truncate"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.RestrictionProvider.truncate">[docs]</a> <span class="k">def</span> <span class="nf">truncate</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Truncates the provided restriction into a restriction representing a</span>
<span class="sd"> finite amount of work when the pipeline is</span>
<span class="sd"> `draining &lt;https://docs.google.com/document/d/1NExwHlj-2q2WUGhSO4jTu8XGhDPmm3cllSN8IMmWci8/edit#&gt; for additional details about drain.&gt;_`. # pylint: disable=line-too-long</span>
<span class="sd"> By default, if the restriction is bounded then the restriction will be</span>
<span class="sd"> returned otherwise None will be returned.</span>
<span class="sd"> This API is optional and should only be implemented if more granularity is</span>
<span class="sd"> required.</span>
<span class="sd"> Return a truncated finite restriction if further processing is required</span>
<span class="sd"> otherwise return None to represent that no further processing of this</span>
<span class="sd"> restriction is required.</span>
<span class="sd"> The default behavior when a pipeline is being drained is that bounded</span>
<span class="sd"> restrictions process entirely while unbounded restrictions process till a</span>
<span class="sd"> checkpoint is possible.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">restriction_tracker</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">create_tracker</span><span class="p">(</span><span class="n">restriction</span><span class="p">)</span>
<span class="k">if</span> <span class="n">restriction_tracker</span><span class="o">.</span><span class="n">is_bounded</span><span class="p">():</span>
<span class="k">return</span> <span class="n">restriction</span></div></div>
<span class="k">def</span> <span class="nf">get_function_arguments</span><span class="p">(</span><span class="n">obj</span><span class="p">,</span> <span class="n">func</span><span class="p">):</span>
<span class="c1"># type: (...) -&gt; typing.Tuple[typing.List[str], typing.List[typing.Any]]</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Return the function arguments based on the name provided. If they have</span>
<span class="sd"> a _inspect_function attached to the class then use that otherwise default</span>
<span class="sd"> to the modified version of python inspect library.</span>
<span class="sd"> Returns:</span>
<span class="sd"> Same as get_function_args_defaults.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">func_name</span> <span class="o">=</span> <span class="s1">&#39;_inspect_</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">func</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">obj</span><span class="p">,</span> <span class="n">func_name</span><span class="p">):</span>
<span class="n">f</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">obj</span><span class="p">,</span> <span class="n">func_name</span><span class="p">)</span>
<span class="k">return</span> <span class="n">f</span><span class="p">()</span>
<span class="n">f</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">obj</span><span class="p">,</span> <span class="n">func</span><span class="p">)</span>
<span class="k">return</span> <span class="n">get_function_args_defaults</span><span class="p">(</span><span class="n">f</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_function_args_defaults</span><span class="p">(</span><span class="n">f</span><span class="p">):</span>
<span class="c1"># type: (...) -&gt; typing.Tuple[typing.List[str], typing.List[typing.Any]]</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns the function arguments of a given function.</span>
<span class="sd"> Returns:</span>
<span class="sd"> (args: List[str], defaults: List[Any]). The first list names the</span>
<span class="sd"> arguments of the method and the second one has the values of the default</span>
<span class="sd"> arguments. This is similar to ``inspect.getfullargspec()``&#39;s results, except</span>
<span class="sd"> it doesn&#39;t include bound arguments and may follow function wrappers.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">signature</span> <span class="o">=</span> <span class="n">get_signature</span><span class="p">(</span><span class="n">f</span><span class="p">)</span>
<span class="n">parameter</span> <span class="o">=</span> <span class="n">inspect</span><span class="o">.</span><span class="n">Parameter</span>
<span class="c1"># TODO(BEAM-5878) support kwonlyargs on Python 3.</span>
<span class="n">_SUPPORTED_ARG_TYPES</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">parameter</span><span class="o">.</span><span class="n">POSITIONAL_ONLY</span><span class="p">,</span> <span class="n">parameter</span><span class="o">.</span><span class="n">POSITIONAL_OR_KEYWORD</span>
<span class="p">]</span>
<span class="n">args</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">name</span> <span class="k">for</span> <span class="n">name</span><span class="p">,</span>
<span class="n">p</span> <span class="ow">in</span> <span class="n">signature</span><span class="o">.</span><span class="n">parameters</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> <span class="k">if</span> <span class="n">p</span><span class="o">.</span><span class="n">kind</span> <span class="ow">in</span> <span class="n">_SUPPORTED_ARG_TYPES</span>
<span class="p">]</span>
<span class="n">defaults</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">p</span><span class="o">.</span><span class="n">default</span> <span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="n">signature</span><span class="o">.</span><span class="n">parameters</span><span class="o">.</span><span class="n">values</span><span class="p">()</span>
<span class="k">if</span> <span class="n">p</span><span class="o">.</span><span class="n">kind</span> <span class="ow">in</span> <span class="n">_SUPPORTED_ARG_TYPES</span> <span class="ow">and</span> <span class="n">p</span><span class="o">.</span><span class="n">default</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">p</span><span class="o">.</span><span class="n">empty</span>
<span class="p">]</span>
<span class="k">return</span> <span class="n">args</span><span class="p">,</span> <span class="n">defaults</span>
<div class="viewcode-block" id="WatermarkEstimatorProvider"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WatermarkEstimatorProvider">[docs]</a><span class="k">class</span> <span class="nc">WatermarkEstimatorProvider</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Provides methods for generating WatermarkEstimator.</span>
<span class="sd"> This class should be implemented if wanting to providing output_watermark</span>
<span class="sd"> information within an SDF.</span>
<span class="sd"> In order to make an SDF.process() access to the typical WatermarkEstimator,</span>
<span class="sd"> the SDF author should have an argument whose default value is a</span>
<span class="sd"> DoFn.WatermarkEstimatorParam instance. This DoFn.WatermarkEstimatorParam</span>
<span class="sd"> can either be constructed with an explicit WatermarkEstimatorProvider,</span>
<span class="sd"> or, if no WatermarkEstimatorProvider is provided, the DoFn itself must</span>
<span class="sd"> be a WatermarkEstimatorProvider.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="WatermarkEstimatorProvider.initial_estimator_state"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WatermarkEstimatorProvider.initial_estimator_state">[docs]</a> <span class="k">def</span> <span class="nf">initial_estimator_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns the initial state of the WatermarkEstimator with given element</span>
<span class="sd"> and restriction.</span>
<span class="sd"> This function is called by the system.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="WatermarkEstimatorProvider.create_watermark_estimator"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WatermarkEstimatorProvider.create_watermark_estimator">[docs]</a> <span class="k">def</span> <span class="nf">create_watermark_estimator</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">estimator_state</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Create a new WatermarkEstimator based on the state. The state is</span>
<span class="sd"> typically useful when resuming processing an element.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="WatermarkEstimatorProvider.estimator_state_coder"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WatermarkEstimatorProvider.estimator_state_coder">[docs]</a> <span class="k">def</span> <span class="nf">estimator_state_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">registry</span><span class="o">.</span><span class="n">get_coder</span><span class="p">(</span><span class="nb">object</span><span class="p">)</span></div></div>
<span class="k">class</span> <span class="nc">_DoFnParam</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;DoFn parameter.&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">param_id</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">param_id</span> <span class="o">=</span> <span class="n">param_id</span>
<span class="k">def</span> <span class="fm">__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">type</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">==</span> <span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">param_id</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">param_id</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">def</span> <span class="fm">__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">param_id</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__repr__</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">param_id</span>
<span class="k">class</span> <span class="nc">_RestrictionDoFnParam</span><span class="p">(</span><span class="n">_DoFnParam</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Restriction Provider DoFn parameter.&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">restriction_provider</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="c1"># type: (typing.Optional[RestrictionProvider]) -&gt; None</span>
<span class="k">if</span> <span class="p">(</span><span class="n">restriction_provider</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span>
<span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">restriction_provider</span><span class="p">,</span> <span class="n">RestrictionProvider</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;DoFn.RestrictionParam expected RestrictionProvider object.&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">restriction_provider</span> <span class="o">=</span> <span class="n">restriction_provider</span>
<span class="bp">self</span><span class="o">.</span><span class="n">param_id</span> <span class="o">=</span> <span class="p">(</span>
<span class="s1">&#39;RestrictionParam(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">restriction_provider</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="k">class</span> <span class="nc">_StateDoFnParam</span><span class="p">(</span><span class="n">_DoFnParam</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;State DoFn parameter.&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">state_spec</span><span class="p">):</span>
<span class="c1"># type: (StateSpec) -&gt; None</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">state_spec</span><span class="p">,</span> <span class="n">StateSpec</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;DoFn.StateParam expected StateSpec object.&quot;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state_spec</span> <span class="o">=</span> <span class="n">state_spec</span>
<span class="bp">self</span><span class="o">.</span><span class="n">param_id</span> <span class="o">=</span> <span class="s1">&#39;StateParam(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">state_spec</span><span class="o">.</span><span class="n">name</span>
<span class="k">class</span> <span class="nc">_TimerDoFnParam</span><span class="p">(</span><span class="n">_DoFnParam</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Timer DoFn parameter.&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">timer_spec</span><span class="p">):</span>
<span class="c1"># type: (TimerSpec) -&gt; None</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">timer_spec</span><span class="p">,</span> <span class="n">TimerSpec</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;DoFn.TimerParam expected TimerSpec object.&quot;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timer_spec</span> <span class="o">=</span> <span class="n">timer_spec</span>
<span class="bp">self</span><span class="o">.</span><span class="n">param_id</span> <span class="o">=</span> <span class="s1">&#39;TimerParam(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">timer_spec</span><span class="o">.</span><span class="n">name</span>
<span class="k">class</span> <span class="nc">_BundleFinalizerParam</span><span class="p">(</span><span class="n">_DoFnParam</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Bundle Finalization DoFn parameter.&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="bp">self</span><span class="o">.</span><span class="n">_callbacks</span> <span class="o">=</span> <span class="p">[]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">param_id</span> <span class="o">=</span> <span class="s2">&quot;FinalizeBundle&quot;</span>
<span class="k">def</span> <span class="nf">register</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">callback</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_callbacks</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">callback</span><span class="p">)</span>
<span class="c1"># Log errors when calling callback to make sure all callbacks get called</span>
<span class="c1"># though there are errors. And errors should not fail pipeline.</span>
<span class="k">def</span> <span class="nf">finalize_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">for</span> <span class="n">callback</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_callbacks</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">callback</span><span class="p">()</span>
<span class="k">except</span> <span class="ne">Exception</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span><span class="s2">&quot;Got exception from finalization call: </span><span class="si">%s</span><span class="s2">&quot;</span><span class="p">,</span> <span class="n">e</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">has_callbacks</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; bool</span>
<span class="k">return</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_callbacks</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span>
<span class="k">def</span> <span class="nf">reset</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; None</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_callbacks</span><span class="p">[:]</span>
<span class="k">class</span> <span class="nc">_WatermarkEstimatorParam</span><span class="p">(</span><span class="n">_DoFnParam</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;WatermarkEstimator DoFn parameter.&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">watermark_estimator_provider</span><span class="p">:</span> <span class="n">typing</span><span class="o">.</span>
<span class="n">Optional</span><span class="p">[</span><span class="n">WatermarkEstimatorProvider</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">):</span>
<span class="k">if</span> <span class="p">(</span><span class="n">watermark_estimator_provider</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">watermark_estimator_provider</span><span class="p">,</span> <span class="n">WatermarkEstimatorProvider</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;DoFn.WatermarkEstimatorParam expected&#39;</span>
<span class="s1">&#39;WatermarkEstimatorProvider object.&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">watermark_estimator_provider</span> <span class="o">=</span> <span class="n">watermark_estimator_provider</span>
<span class="bp">self</span><span class="o">.</span><span class="n">param_id</span> <span class="o">=</span> <span class="s1">&#39;WatermarkEstimatorProvider&#39;</span>
<div class="viewcode-block" id="DoFn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn">[docs]</a><span class="k">class</span> <span class="nc">DoFn</span><span class="p">(</span><span class="n">WithTypeHints</span><span class="p">,</span> <span class="n">HasDisplayData</span><span class="p">,</span> <span class="n">urns</span><span class="o">.</span><span class="n">RunnerApiFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A function object used by a transform with custom processing.</span>
<span class="sd"> The ParDo transform is such a transform. The ParDo.apply</span>
<span class="sd"> method will take an object of type DoFn and apply it to all elements of a</span>
<span class="sd"> PCollection object.</span>
<span class="sd"> In order to have concrete DoFn objects one has to subclass from DoFn and</span>
<span class="sd"> define the desired behavior (start_bundle/finish_bundle and process) or wrap a</span>
<span class="sd"> callable object using the CallableWrapperDoFn class.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Parameters that can be used in the .process() method.</span>
<span class="n">ElementParam</span> <span class="o">=</span> <span class="n">_DoFnParam</span><span class="p">(</span><span class="s1">&#39;ElementParam&#39;</span><span class="p">)</span>
<span class="n">SideInputParam</span> <span class="o">=</span> <span class="n">_DoFnParam</span><span class="p">(</span><span class="s1">&#39;SideInputParam&#39;</span><span class="p">)</span>
<span class="n">TimestampParam</span> <span class="o">=</span> <span class="n">_DoFnParam</span><span class="p">(</span><span class="s1">&#39;TimestampParam&#39;</span><span class="p">)</span>
<span class="n">WindowParam</span> <span class="o">=</span> <span class="n">_DoFnParam</span><span class="p">(</span><span class="s1">&#39;WindowParam&#39;</span><span class="p">)</span>
<span class="n">PaneInfoParam</span> <span class="o">=</span> <span class="n">_DoFnParam</span><span class="p">(</span><span class="s1">&#39;PaneInfoParam&#39;</span><span class="p">)</span>
<span class="n">WatermarkEstimatorParam</span> <span class="o">=</span> <span class="n">_WatermarkEstimatorParam</span>
<span class="n">BundleFinalizerParam</span> <span class="o">=</span> <span class="n">_BundleFinalizerParam</span>
<span class="n">KeyParam</span> <span class="o">=</span> <span class="n">_DoFnParam</span><span class="p">(</span><span class="s1">&#39;KeyParam&#39;</span><span class="p">)</span>
<span class="c1"># Parameters to access state and timers. Not restricted to use only in the</span>
<span class="c1"># .process() method. Usage: DoFn.StateParam(state_spec),</span>
<span class="c1"># DoFn.TimerParam(timer_spec), DoFn.TimestampParam, DoFn.WindowParam,</span>
<span class="c1"># DoFn.KeyParam</span>
<span class="n">StateParam</span> <span class="o">=</span> <span class="n">_StateDoFnParam</span>
<span class="n">TimerParam</span> <span class="o">=</span> <span class="n">_TimerDoFnParam</span>
<span class="n">DynamicTimerTagParam</span> <span class="o">=</span> <span class="n">_DoFnParam</span><span class="p">(</span><span class="s1">&#39;DynamicTimerTagParam&#39;</span><span class="p">)</span>
<span class="n">DoFnProcessParams</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">ElementParam</span><span class="p">,</span>
<span class="n">SideInputParam</span><span class="p">,</span>
<span class="n">TimestampParam</span><span class="p">,</span>
<span class="n">WindowParam</span><span class="p">,</span>
<span class="n">WatermarkEstimatorParam</span><span class="p">,</span>
<span class="n">PaneInfoParam</span><span class="p">,</span>
<span class="n">BundleFinalizerParam</span><span class="p">,</span>
<span class="n">KeyParam</span><span class="p">,</span>
<span class="n">StateParam</span><span class="p">,</span>
<span class="n">TimerParam</span><span class="p">,</span>
<span class="p">]</span>
<span class="n">RestrictionParam</span> <span class="o">=</span> <span class="n">_RestrictionDoFnParam</span>
<div class="viewcode-block" id="DoFn.from_callable"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.from_callable">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">return</span> <span class="n">CallableWrapperDoFn</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span></div>
<div class="viewcode-block" id="DoFn.unbounded_per_element"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.unbounded_per_element">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">unbounded_per_element</span><span class="p">():</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A decorator on process fn specifying that the fn performs an unbounded</span>
<span class="sd"> amount of work per input element.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">wrapper</span><span class="p">(</span><span class="n">process_fn</span><span class="p">):</span>
<span class="n">process_fn</span><span class="o">.</span><span class="n">unbounded_per_element</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">return</span> <span class="n">process_fn</span>
<span class="k">return</span> <span class="n">wrapper</span></div>
<div class="viewcode-block" id="DoFn.yields_elements"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.yields_elements">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">yields_elements</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A decorator to apply to ``process_batch`` indicating it yields elements.</span>
<span class="sd"> By default ``process_batch`` is assumed to both consume and produce</span>
<span class="sd"> &quot;batches&quot;, which are collections of multiple logical Beam elements. This</span>
<span class="sd"> decorator indicates that ``process_batch`` **produces** individual elements</span>
<span class="sd"> at a time. ``process_batch`` is always expected to consume batches.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">fn</span><span class="o">.</span><span class="vm">__name__</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">&#39;process&#39;</span><span class="p">,</span> <span class="s1">&#39;process_batch&#39;</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;@yields_elements must be applied to a process or &quot;</span>
<span class="sa">f</span><span class="s2">&quot;process_batch method, got </span><span class="si">{</span><span class="n">fn</span><span class="si">!r}</span><span class="s2">.&quot;</span><span class="p">)</span>
<span class="n">fn</span><span class="o">.</span><span class="n">_beam_yields_elements</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">return</span> <span class="n">fn</span></div>
<div class="viewcode-block" id="DoFn.yields_batches"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.yields_batches">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">yields_batches</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A decorator to apply to ``process`` indicating it yields batches.</span>
<span class="sd"> By default ``process`` is assumed to both consume and produce</span>
<span class="sd"> individual elements at a time. This decorator indicates that ``process``</span>
<span class="sd"> **produces** &quot;batches&quot;, which are collections of multiple logical Beam</span>
<span class="sd"> elements.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">fn</span><span class="o">.</span><span class="vm">__name__</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">&#39;process&#39;</span><span class="p">,</span> <span class="s1">&#39;process_batch&#39;</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;@yields_elements must be applied to a process or &quot;</span>
<span class="sa">f</span><span class="s2">&quot;process_batch method, got </span><span class="si">{</span><span class="n">fn</span><span class="si">!r}</span><span class="s2">.&quot;</span><span class="p">)</span>
<span class="n">fn</span><span class="o">.</span><span class="n">_beam_yields_batches</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">return</span> <span class="n">fn</span></div>
<div class="viewcode-block" id="DoFn.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.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="bp">self</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span></div>
<div class="viewcode-block" id="DoFn.process"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.process">[docs]</a> <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="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="w"> </span><span class="sd">&quot;&quot;&quot;Method to use for processing elements.</span>
<span class="sd"> This is invoked by ``DoFnRunner`` for each element of a input</span>
<span class="sd"> ``PCollection``.</span>
<span class="sd"> The following parameters can be used as default values on ``process``</span>
<span class="sd"> arguments to indicate that a DoFn accepts the corresponding parameters. For</span>
<span class="sd"> example, a DoFn might accept the element and its timestamp with the</span>
<span class="sd"> following signature::</span>
<span class="sd"> def process(element=DoFn.ElementParam, timestamp=DoFn.TimestampParam):</span>
<span class="sd"> ...</span>
<span class="sd"> The full set of parameters is:</span>
<span class="sd"> - ``DoFn.ElementParam``: element to be processed, should not be mutated.</span>
<span class="sd"> - ``DoFn.SideInputParam``: a side input that may be used when processing.</span>
<span class="sd"> - ``DoFn.TimestampParam``: timestamp of the input element.</span>
<span class="sd"> - ``DoFn.WindowParam``: ``Window`` the input element belongs to.</span>
<span class="sd"> - ``DoFn.TimerParam``: a ``userstate.RuntimeTimer`` object defined by the</span>
<span class="sd"> spec of the parameter.</span>
<span class="sd"> - ``DoFn.StateParam``: a ``userstate.RuntimeState`` object defined by the</span>
<span class="sd"> spec of the parameter.</span>
<span class="sd"> - ``DoFn.KeyParam``: key associated with the element.</span>
<span class="sd"> - ``DoFn.RestrictionParam``: an ``iobase.RestrictionTracker`` will be</span>
<span class="sd"> provided here to allow treatment as a Splittable ``DoFn``. The restriction</span>
<span class="sd"> tracker will be derived from the restriction provider in the parameter.</span>
<span class="sd"> - ``DoFn.WatermarkEstimatorParam``: a function that can be used to track</span>
<span class="sd"> output watermark of Splittable ``DoFn`` implementations.</span>
<span class="sd"> Args:</span>
<span class="sd"> element: The element to be processed</span>
<span class="sd"> *args: side inputs</span>
<span class="sd"> **kwargs: other keyword arguments.</span>
<span class="sd"> Returns:</span>
<span class="sd"> An Iterable of output elements or None.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="DoFn.process_batch"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.process_batch">[docs]</a> <span class="k">def</span> <span class="nf">process_batch</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">batch</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">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="DoFn.setup"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.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="w"> </span><span class="sd">&quot;&quot;&quot;Called to prepare an instance for processing bundles of elements.</span>
<span class="sd"> This is a good place to initialize transient in-memory resources, such as</span>
<span class="sd"> network connections. The resources can then be disposed in</span>
<span class="sd"> ``DoFn.teardown``.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="DoFn.start_bundle"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.start_bundle">[docs]</a> <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="w"> </span><span class="sd">&quot;&quot;&quot;Called before a bundle of elements is processed on a worker.</span>
<span class="sd"> Elements to be processed are split into bundles and distributed</span>
<span class="sd"> to workers. Before a worker calls process() on the first element</span>
<span class="sd"> of its bundle, it calls this method.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="DoFn.finish_bundle"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.finish_bundle">[docs]</a> <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="w"> </span><span class="sd">&quot;&quot;&quot;Called after a bundle of elements is processed on a worker.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="DoFn.teardown"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.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="w"> </span><span class="sd">&quot;&quot;&quot;Called to use to clean up this instance before it is discarded.</span>
<span class="sd"> A runner will do its best to call this method on any given instance to</span>
<span class="sd"> prevent leaks of transient resources, however, there may be situations where</span>
<span class="sd"> this is impossible (e.g. process crash, hardware failure, etc.) or</span>
<span class="sd"> unnecessary (e.g. the pipeline is shutting down and the process is about to</span>
<span class="sd"> be killed anyway, so all transient resources will be released automatically</span>
<span class="sd"> by the OS). In these cases, the call may not happen. It will also not be</span>
<span class="sd"> retried, because in such situations the DoFn instance no longer exists, so</span>
<span class="sd"> there&#39;s no instance to retry it on.</span>
<span class="sd"> Thus, all work that depends on input elements, and all externally important</span>
<span class="sd"> side effects, must be performed in ``DoFn.process`` or</span>
<span class="sd"> ``DoFn.finish_bundle``.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="DoFn.get_function_arguments"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.get_function_arguments">[docs]</a> <span class="k">def</span> <span class="nf">get_function_arguments</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">):</span>
<span class="k">return</span> <span class="n">get_function_arguments</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">)</span></div>
<div class="viewcode-block" id="DoFn.default_type_hints"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.default_type_hints">[docs]</a> <span class="k">def</span> <span class="nf">default_type_hints</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">process_type_hints</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process</span><span class="p">)</span> <span class="ow">or</span> <span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">empty</span><span class="p">()</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_yields_batches</span><span class="p">:</span>
<span class="c1"># process() produces batches, don&#39;t use it&#39;s output typehint</span>
<span class="n">process_type_hints</span> <span class="o">=</span> <span class="n">process_type_hints</span><span class="o">.</span><span class="n">with_output_types_from</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">empty</span><span class="p">())</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_batch_yields_elements</span><span class="p">:</span>
<span class="c1"># process_batch() produces elements, *do* use it&#39;s output typehint</span>
<span class="c1"># First access the typehint</span>
<span class="n">process_batch_type_hints</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process_batch</span><span class="p">)</span> <span class="ow">or</span> <span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">empty</span><span class="p">()</span>
<span class="c1"># Then we deconflict with the typehint from process, if it exists</span>
<span class="k">if</span> <span class="p">(</span><span class="n">process_batch_type_hints</span><span class="o">.</span><span class="n">output_types</span> <span class="o">!=</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">empty</span><span class="p">()</span><span class="o">.</span><span class="n">output_types</span><span class="p">):</span>
<span class="k">if</span> <span class="p">(</span><span class="n">process_type_hints</span><span class="o">.</span><span class="n">output_types</span> <span class="o">!=</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">empty</span><span class="p">()</span><span class="o">.</span><span class="n">output_types</span> <span class="ow">and</span>
<span class="n">process_batch_type_hints</span><span class="o">.</span><span class="n">output_types</span> <span class="o">!=</span>
<span class="n">process_type_hints</span><span class="o">.</span><span class="n">output_types</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="sa">f</span><span class="s2">&quot;DoFn </span><span class="si">{</span><span class="bp">self</span><span class="si">!r}</span><span class="s2"> yields element from both process and &quot;</span>
<span class="s2">&quot;process_batch, but they have mismatched output typehints:</span><span class="se">\n</span><span class="s2">&quot;</span>
<span class="sa">f</span><span class="s2">&quot; process: </span><span class="si">{</span><span class="n">process_type_hints</span><span class="o">.</span><span class="n">output_types</span><span class="si">}</span><span class="se">\n</span><span class="s2">&quot;</span>
<span class="sa">f</span><span class="s2">&quot; process_batch: </span><span class="si">{</span><span class="n">process_batch_type_hints</span><span class="o">.</span><span class="n">output_types</span><span class="si">}</span><span class="s2">&quot;</span><span class="p">)</span>
<span class="n">process_type_hints</span> <span class="o">=</span> <span class="n">process_type_hints</span><span class="o">.</span><span class="n">with_output_types_from</span><span class="p">(</span>
<span class="n">process_batch_type_hints</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">process_type_hints</span> <span class="o">=</span> <span class="n">process_type_hints</span><span class="o">.</span><span class="n">strip_iterable</span><span class="p">()</span>
<span class="k">except</span> <span class="ne">ValueError</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Return value not iterable: </span><span class="si">%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="bp">self</span><span class="p">,</span> <span class="n">e</span><span class="p">))</span>
<span class="c1"># Prefer class decorator type hints for backwards compatibility.</span>
<span class="k">return</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="vm">__class__</span><span class="p">)</span><span class="o">.</span><span class="n">with_defaults</span><span class="p">(</span><span class="n">process_type_hints</span><span class="p">)</span></div>
<span class="c1"># TODO(sourabhbajaj): Do we want to remove the responsibility of these from</span>
<span class="c1"># the DoFn or maybe the runner</span>
<div class="viewcode-block" id="DoFn.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_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="c1"># TODO(https://github.com/apache/beam/issues/19824): Side inputs types.</span>
<span class="k">return</span> <span class="n">trivial_inference</span><span class="o">.</span><span class="n">element_type</span><span class="p">(</span>
<span class="n">_strip_output_annotations</span><span class="p">(</span>
<span class="n">trivial_inference</span><span class="o">.</span><span class="n">infer_return_type</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">process</span><span class="p">,</span> <span class="p">[</span><span class="n">input_type</span><span class="p">])))</span></div>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_process_defined</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">bool</span><span class="p">:</span>
<span class="c1"># Check if this DoFn&#39;s process method has been overridden</span>
<span class="c1"># Note that we retrieve the __func__ attribute, if it exists, to get the</span>
<span class="c1"># underlying function from the bound method.</span>
<span class="c1"># If __func__ doesn&#39;t exist, self.process was likely overridden with a free</span>
<span class="c1"># function, as in CallableWrapperDoFn.</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">process</span><span class="p">,</span> <span class="s1">&#39;__func__&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">process</span><span class="p">)</span> <span class="o">!=</span> <span class="n">DoFn</span><span class="o">.</span><span class="n">process</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_process_batch_defined</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">bool</span><span class="p">:</span>
<span class="c1"># Check if this DoFn&#39;s process_batch method has been overridden</span>
<span class="c1"># Note that we retrieve the __func__ attribute, if it exists, to get the</span>
<span class="c1"># underlying function from the bound method.</span>
<span class="c1"># If __func__ doesn&#39;t exist, self.process_batch was likely overridden with</span>
<span class="c1"># a free function.</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process_batch</span><span class="p">,</span> <span class="s1">&#39;__func__&#39;</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process_batch</span><span class="p">)</span> <span class="o">!=</span> <span class="n">DoFn</span><span class="o">.</span><span class="n">process_batch</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_can_yield_batches</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">bool</span><span class="p">:</span>
<span class="k">return</span> <span class="p">((</span><span class="bp">self</span><span class="o">.</span><span class="n">_process_defined</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_yields_batches</span><span class="p">)</span> <span class="ow">or</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_batch_defined</span> <span class="ow">and</span>
<span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_batch_yields_elements</span><span class="p">))</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_process_yields_batches</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">bool</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">process</span><span class="p">,</span> <span class="s1">&#39;_beam_yields_batches&#39;</span><span class="p">,</span> <span class="kc">False</span><span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_process_batch_yields_elements</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">bool</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">process_batch</span><span class="p">,</span> <span class="s1">&#39;_beam_yields_elements&#39;</span><span class="p">,</span> <span class="kc">False</span><span class="p">)</span>
<div class="viewcode-block" id="DoFn.get_input_batch_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.get_input_batch_type">[docs]</a> <span class="k">def</span> <span class="nf">get_input_batch_type</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">input_element_type</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">typing</span><span class="o">.</span><span class="n">Optional</span><span class="p">[</span><span class="n">typing</span><span class="o">.</span><span class="n">Union</span><span class="p">[</span><span class="n">TypeConstraint</span><span class="p">,</span> <span class="nb">type</span><span class="p">]]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Determine the batch type expected as input to process_batch.</span>
<span class="sd"> The default implementation of ``get_input_batch_type`` simply observes the</span>
<span class="sd"> input typehint for the first parameter of ``process_batch``. A Batched DoFn</span>
<span class="sd"> may override this method if a dynamic approach is required.</span>
<span class="sd"> Args:</span>
<span class="sd"> input_element_type: The **element type** of the input PCollection this</span>
<span class="sd"> DoFn is being applied to.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ``None`` if this DoFn cannot accept batches, else a Beam typehint or</span>
<span class="sd"> a native Python typehint.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_batch_defined</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="n">input_type</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span>
<span class="n">inspect</span><span class="o">.</span><span class="n">signature</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">process_batch</span><span class="p">)</span><span class="o">.</span><span class="n">parameters</span><span class="o">.</span><span class="n">values</span><span class="p">())[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">annotation</span>
<span class="k">if</span> <span class="n">input_type</span> <span class="o">==</span> <span class="n">inspect</span><span class="o">.</span><span class="n">Signature</span><span class="o">.</span><span class="n">empty</span><span class="p">:</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/21652): Consider supporting</span>
<span class="c1"># an alternative (dynamic?) approach for declaring input type</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="sa">f</span><span class="s2">&quot;Either </span><span class="si">{</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="si">}</span><span class="s2">.process_batch() must have a type &quot;</span>
<span class="sa">f</span><span class="s2">&quot;annotation on its first parameter, or </span><span class="si">{</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="si">}</span><span class="s2"> &quot;</span>
<span class="s2">&quot;must override get_input_batch_type.&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">input_type</span></div>
<span class="k">def</span> <span class="nf">_get_input_batch_type_normalized</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">input_element_type</span><span class="p">):</span>
<span class="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">native_type_compatibility</span><span class="o">.</span><span class="n">convert_to_beam_type</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">get_input_batch_type</span><span class="p">(</span><span class="n">input_element_type</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">_get_output_batch_type_normalized</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">input_element_type</span><span class="p">):</span>
<span class="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">native_type_compatibility</span><span class="o">.</span><span class="n">convert_to_beam_type</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">get_output_batch_type</span><span class="p">(</span><span class="n">input_element_type</span><span class="p">))</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_get_element_type_from_return_annotation</span><span class="p">(</span><span class="n">method</span><span class="p">,</span> <span class="n">input_type</span><span class="p">):</span>
<span class="n">return_type</span> <span class="o">=</span> <span class="n">inspect</span><span class="o">.</span><span class="n">signature</span><span class="p">(</span><span class="n">method</span><span class="p">)</span><span class="o">.</span><span class="n">return_annotation</span>
<span class="k">if</span> <span class="n">return_type</span> <span class="o">==</span> <span class="n">inspect</span><span class="o">.</span><span class="n">Signature</span><span class="o">.</span><span class="n">empty</span><span class="p">:</span>
<span class="c1"># output type not annotated, try to infer it</span>
<span class="n">return_type</span> <span class="o">=</span> <span class="n">trivial_inference</span><span class="o">.</span><span class="n">infer_return_type</span><span class="p">(</span><span class="n">method</span><span class="p">,</span> <span class="p">[</span><span class="n">input_type</span><span class="p">])</span>
<span class="n">return_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">native_type_compatibility</span><span class="o">.</span><span class="n">convert_to_beam_type</span><span class="p">(</span>
<span class="n">return_type</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">return_type</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">typehints</span><span class="o">.</span><span class="n">IterableTypeConstraint</span><span class="p">):</span>
<span class="k">return</span> <span class="n">return_type</span><span class="o">.</span><span class="n">inner_type</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">return_type</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">typehints</span><span class="o">.</span><span class="n">IteratorTypeConstraint</span><span class="p">):</span>
<span class="k">return</span> <span class="n">return_type</span><span class="o">.</span><span class="n">yielded_type</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Expected Iterator in return type annotation for &quot;</span>
<span class="sa">f</span><span class="s2">&quot;</span><span class="si">{</span><span class="n">method</span><span class="si">!r}</span><span class="s2">, did you mean Iterator[</span><span class="si">{</span><span class="n">return_type</span><span class="si">}</span><span class="s2">]? Note Beam DoFn &quot;</span>
<span class="s2">&quot;process and process_batch methods are expected to produce &quot;</span>
<span class="s2">&quot;generators - they should &#39;yield&#39; rather than &#39;return&#39;.&quot;</span><span class="p">)</span>
<div class="viewcode-block" id="DoFn.get_output_batch_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.DoFn.get_output_batch_type">[docs]</a> <span class="k">def</span> <span class="nf">get_output_batch_type</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">input_element_type</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">typing</span><span class="o">.</span><span class="n">Optional</span><span class="p">[</span><span class="n">typing</span><span class="o">.</span><span class="n">Union</span><span class="p">[</span><span class="n">TypeConstraint</span><span class="p">,</span> <span class="nb">type</span><span class="p">]]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Determine the batch type produced by this DoFn&#39;s ``process_batch``</span>
<span class="sd"> implementation and/or its ``process`` implementation with</span>
<span class="sd"> ``@yields_batch``.</span>
<span class="sd"> The default implementation of this method observes the return type</span>
<span class="sd"> annotations on ``process_batch`` and/or ``process``. A Batched DoFn may</span>
<span class="sd"> override this method if a dynamic approach is required.</span>
<span class="sd"> Args:</span>
<span class="sd"> input_element_type: The **element type** of the input PCollection this</span>
<span class="sd"> DoFn is being applied to.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ``None`` if this DoFn will never yield batches, else a Beam typehint or</span>
<span class="sd"> a native Python typehint.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">output_batch_type</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_defined</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_yields_batches</span><span class="p">:</span>
<span class="n">output_batch_type</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_element_type_from_return_annotation</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process</span><span class="p">,</span> <span class="n">input_element_type</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_batch_defined</span> <span class="ow">and</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_batch_yields_elements</span><span class="p">:</span>
<span class="n">process_batch_type</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_element_type_from_return_annotation</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process_batch</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_get_input_batch_type_normalized</span><span class="p">(</span><span class="n">input_element_type</span><span class="p">))</span>
<span class="c1"># TODO: Consider requiring an inheritance relationship rather than</span>
<span class="c1"># equality</span>
<span class="k">if</span> <span class="p">(</span><span class="n">output_batch_type</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span>
<span class="p">(</span><span class="ow">not</span> <span class="n">process_batch_type</span> <span class="o">==</span> <span class="n">output_batch_type</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="sa">f</span><span class="s2">&quot;DoFn </span><span class="si">{</span><span class="bp">self</span><span class="si">!r}</span><span class="s2"> yields batches from both process and &quot;</span>
<span class="s2">&quot;process_batch, but they produce different types:</span><span class="se">\n</span><span class="s2">&quot;</span>
<span class="sa">f</span><span class="s2">&quot; process: </span><span class="si">{</span><span class="n">output_batch_type</span><span class="si">}</span><span class="se">\n</span><span class="s2">&quot;</span>
<span class="sa">f</span><span class="s2">&quot; process_batch: </span><span class="si">{</span><span class="n">process_batch_type</span><span class="si">!r}</span><span class="s2">&quot;</span><span class="p">)</span>
<span class="n">output_batch_type</span> <span class="o">=</span> <span class="n">process_batch_type</span>
<span class="k">return</span> <span class="n">output_batch_type</span></div>
<span class="k">def</span> <span class="nf">_process_argspec_fn</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns the Python callable that will eventually be invoked.</span>
<span class="sd"> This should ideally be the user-level function that is called with</span>
<span class="sd"> the main and (if any) side inputs, and is used to relate the type</span>
<span class="sd"> hint parameters with the input parameters (e.g., by argument name).</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">process</span>
<span class="n">urns</span><span class="o">.</span><span class="n">RunnerApiFn</span><span class="o">.</span><span class="n">register_pickle_urn</span><span class="p">(</span><span class="n">python_urns</span><span class="o">.</span><span class="n">PICKLED_DOFN</span><span class="p">)</span></div>
<span class="k">class</span> <span class="nc">CallableWrapperDoFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> A DoFn (function) object wrapping a callable object.</span>
<span class="sd"> The purpose of this class is to conveniently wrap simple functions and use</span>
<span class="sd"> them in transforms.</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">fn</span><span class="p">,</span> <span class="n">fullargspec</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Initializes a CallableWrapperDoFn object wrapping a callable.</span>
<span class="sd"> Args:</span>
<span class="sd"> fn: A callable object.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: if fn parameter is not a callable type.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Expected a callable object instead of: </span><span class="si">%r</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">fn</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">_fullargspec</span> <span class="o">=</span> <span class="n">fullargspec</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">fn</span><span class="p">,</span> <span class="p">(</span><span class="n">types</span><span class="o">.</span><span class="n">BuiltinFunctionType</span><span class="p">,</span> <span class="n">types</span><span class="o">.</span><span class="n">MethodType</span><span class="p">,</span> <span class="n">types</span><span class="o">.</span><span class="n">FunctionType</span><span class="p">)):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process</span> <span class="o">=</span> <span class="n">fn</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># For cases such as set / list where fn is callable but not a function</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">element</span><span class="p">:</span> <span class="n">fn</span><span class="p">(</span><span class="n">element</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="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># If the callable has a name, then it&#39;s likely a function, and</span>
<span class="c1"># we show its name.</span>
<span class="c1"># Otherwise, it might be an instance of a callable class. We</span>
<span class="c1"># show its class.</span>
<span class="n">display_data_value</span> <span class="o">=</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="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">_fn</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">_fn</span><span class="o">.</span><span class="vm">__class__</span><span class="p">)</span>
<span class="k">return</span> <span class="p">{</span>
<span class="s1">&#39;fn&#39;</span><span class="p">:</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="n">display_data_value</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Transform Function&#39;</span><span class="p">)</span>
<span class="p">}</span>
<span class="k">def</span> <span class="fm">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;CallableWrapperDoFn(</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">_fn</span>
<span class="k">def</span> <span class="nf">default_type_hints</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">fn_type_hints</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">)</span>
<span class="n">type_hints</span> <span class="o">=</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">)</span><span class="o">.</span><span class="n">with_defaults</span><span class="p">(</span><span class="n">fn_type_hints</span><span class="p">)</span>
<span class="c1"># The fn&#39;s output type should be iterable. Strip off the outer</span>
<span class="c1"># container type due to the &#39;flatten&#39; portion of FlatMap/ParDo.</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">type_hints</span> <span class="o">=</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">strip_iterable</span><span class="p">()</span>
<span class="k">except</span> <span class="ne">ValueError</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="k">raise</span> <span class="n">TypeCheckError</span><span class="p">(</span>
<span class="s1">&#39;Return value not iterable: </span><span class="si">%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="bp">self</span><span class="o">.</span><span class="n">display_data</span><span class="p">()[</span><span class="s1">&#39;fn&#39;</span><span class="p">]</span><span class="o">.</span><span class="n">value</span><span class="p">,</span> <span class="n">e</span><span class="p">))</span>
<span class="k">return</span> <span class="n">type_hints</span>
<span class="k">def</span> <span class="nf">infer_output_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">return</span> <span class="n">trivial_inference</span><span class="o">.</span><span class="n">element_type</span><span class="p">(</span>
<span class="n">_strip_output_annotations</span><span class="p">(</span>
<span class="n">trivial_inference</span><span class="o">.</span><span class="n">infer_return_type</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">,</span> <span class="p">[</span><span class="n">input_type</span><span class="p">])))</span>
<span class="k">def</span> <span class="nf">_process_argspec_fn</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">,</span> <span class="s1">&#39;_argspec_fn&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_inspect_process</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fullargspec</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fullargspec</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">get_function_args_defaults</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_process_argspec_fn</span><span class="p">())</span>
<div class="viewcode-block" id="CombineFn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn">[docs]</a><span class="k">class</span> <span class="nc">CombineFn</span><span class="p">(</span><span class="n">WithTypeHints</span><span class="p">,</span> <span class="n">HasDisplayData</span><span class="p">,</span> <span class="n">urns</span><span class="o">.</span><span class="n">RunnerApiFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A function object used by a Combine transform with custom processing.</span>
<span class="sd"> A CombineFn specifies how multiple values in all or part of a PCollection can</span>
<span class="sd"> be merged into a single value---essentially providing the same kind of</span>
<span class="sd"> information as the arguments to the Python &quot;reduce&quot; builtin (except for the</span>
<span class="sd"> input argument, which is an instance of CombineFnProcessContext). The</span>
<span class="sd"> combining process proceeds as follows:</span>
<span class="sd"> 1. Input values are partitioned into one or more batches.</span>
<span class="sd"> 2. For each batch, the setup method is invoked.</span>
<span class="sd"> 3. For each batch, the create_accumulator method is invoked to create a fresh</span>
<span class="sd"> initial &quot;accumulator&quot; value representing the combination of zero values.</span>
<span class="sd"> 4. For each input value in the batch, the add_input method is invoked to</span>
<span class="sd"> combine more values with the accumulator for that batch.</span>
<span class="sd"> 5. The merge_accumulators method is invoked to combine accumulators from</span>
<span class="sd"> separate batches into a single combined output accumulator value, once all</span>
<span class="sd"> of the accumulators have had all the input value in their batches added to</span>
<span class="sd"> them. This operation is invoked repeatedly, until there is only one</span>
<span class="sd"> accumulator value left.</span>
<span class="sd"> 6. The extract_output operation is invoked on the final accumulator to get</span>
<span class="sd"> the output value.</span>
<span class="sd"> 7. The teardown method is invoked.</span>
<span class="sd"> Note: If this **CombineFn** is used with a transform that has defaults,</span>
<span class="sd"> **apply** will be called with an empty list at expansion time to get the</span>
<span class="sd"> default value.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="CombineFn.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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="bp">self</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span></div>
<div class="viewcode-block" id="CombineFn.setup"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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="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="w"> </span><span class="sd">&quot;&quot;&quot;Called to prepare an instance for combining.</span>
<span class="sd"> This method can be useful if there is some state that needs to be loaded</span>
<span class="sd"> before executing any of the other methods. The resources can then be</span>
<span class="sd"> disposed of in ``CombineFn.teardown``.</span>
<span class="sd"> If you are using Dataflow, you need to enable Dataflow Runner V2</span>
<span class="sd"> before using this feature.</span>
<span class="sd"> Args:</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="CombineFn.create_accumulator"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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="w"> </span><span class="sd">&quot;&quot;&quot;Return a fresh, empty accumulator for the combine operation.</span>
<span class="sd"> Args:</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span></div>
<div class="viewcode-block" id="CombineFn.add_input"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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">mutable_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="w"> </span><span class="sd">&quot;&quot;&quot;Return result of folding element into accumulator.</span>
<span class="sd"> CombineFn implementors must override add_input.</span>
<span class="sd"> Args:</span>
<span class="sd"> mutable_accumulator: the current accumulator,</span>
<span class="sd"> may be modified and returned for efficiency</span>
<span class="sd"> element: the element to add, should not be mutated</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span></div>
<div class="viewcode-block" id="CombineFn.add_inputs"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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">mutable_accumulator</span><span class="p">,</span> <span class="n">elements</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="w"> </span><span class="sd">&quot;&quot;&quot;Returns the result of folding each element in elements into accumulator.</span>
<span class="sd"> This is provided in case the implementation affords more efficient</span>
<span class="sd"> bulk addition of elements. The default implementation simply loops</span>
<span class="sd"> over the inputs invoking add_input for each one.</span>
<span class="sd"> Args:</span>
<span class="sd"> mutable_accumulator: the current accumulator,</span>
<span class="sd"> may be modified and returned for efficiency</span>
<span class="sd"> elements: the elements to add, should not be mutated</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">for</span> <span class="n">element</span> <span class="ow">in</span> <span class="n">elements</span><span class="p">:</span>
<span class="n">mutable_accumulator</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">mutable_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="n">mutable_accumulator</span></div>
<div class="viewcode-block" id="CombineFn.merge_accumulators"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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="w"> </span><span class="sd">&quot;&quot;&quot;Returns the result of merging several accumulators</span>
<span class="sd"> to a single accumulator value.</span>
<span class="sd"> Args:</span>
<span class="sd"> accumulators: the accumulators to merge.</span>
<span class="sd"> Only the first accumulator may be modified and returned for efficiency;</span>
<span class="sd"> the other accumulators should not be mutated, because they may be</span>
<span class="sd"> shared with other code and mutating them could lead to incorrect</span>
<span class="sd"> results or data corruption.</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span></div>
<div class="viewcode-block" id="CombineFn.compact"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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="w"> </span><span class="sd">&quot;&quot;&quot;Optionally returns a more compact represenation of the accumulator.</span>
<span class="sd"> This is called before an accumulator is sent across the wire, and can</span>
<span class="sd"> be useful in cases where values are buffered or otherwise lazily</span>
<span class="sd"> kept unprocessed when added to the accumulator. Should return an</span>
<span class="sd"> equivalent, though possibly modified, accumulator.</span>
<span class="sd"> By default returns the accumulator unmodified.</span>
<span class="sd"> Args:</span>
<span class="sd"> accumulator: the current accumulator</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">accumulator</span></div>
<div class="viewcode-block" id="CombineFn.extract_output"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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="w"> </span><span class="sd">&quot;&quot;&quot;Return result of converting accumulator into the output value.</span>
<span class="sd"> Args:</span>
<span class="sd"> accumulator: the final accumulator value computed by this CombineFn</span>
<span class="sd"> for the entire input key or PCollection. Can be modified for</span>
<span class="sd"> efficiency.</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span></div>
<div class="viewcode-block" id="CombineFn.teardown"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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="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="w"> </span><span class="sd">&quot;&quot;&quot;Called to clean up an instance before it is discarded.</span>
<span class="sd"> If you are using Dataflow, you need to enable Dataflow Runner V2</span>
<span class="sd"> before using this feature.</span>
<span class="sd"> Args:</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="CombineFn.apply"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.apply">[docs]</a> <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="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="w"> </span><span class="sd">&quot;&quot;&quot;Returns result of applying this CombineFn to the input values.</span>
<span class="sd"> Args:</span>
<span class="sd"> elements: the set of values to combine.</span>
<span class="sd"> *args: Additional arguments and side inputs.</span>
<span class="sd"> **kwargs: Additional arguments and side inputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">extract_output</span><span class="p">(</span>
<span class="bp">self</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">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="n">elements</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="o">*</span><span class="n">args</span><span class="p">,</span>
<span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombineFn.for_input_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.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="w"> </span><span class="sd">&quot;&quot;&quot;Returns a specialized implementation of self, if it exists.</span>
<span class="sd"> Otherwise, returns self.</span>
<span class="sd"> Args:</span>
<span class="sd"> input_type: the type of input elements.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span></div>
<div class="viewcode-block" id="CombineFn.from_callable"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.from_callable">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">return</span> <span class="n">CallableWrapperCombineFn</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombineFn.maybe_from_callable"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.maybe_from_callable">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">maybe_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">has_side_inputs</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span>
<span class="c1"># type: (typing.Union[CombineFn, typing.Callable], bool) -&gt; CombineFn</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">CombineFn</span><span class="p">):</span>
<span class="k">return</span> <span class="n">fn</span>
<span class="k">elif</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">has_side_inputs</span><span class="p">:</span>
<span class="k">return</span> <span class="n">NoSideInputsCallableWrapperCombineFn</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">return</span> <span class="n">CallableWrapperCombineFn</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Expected a CombineFn or callable, got </span><span class="si">%r</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">fn</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombineFn.get_accumulator_coder"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineFn.get_accumulator_coder">[docs]</a> <span class="k">def</span> <span class="nf">get_accumulator_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">registry</span><span class="o">.</span><span class="n">get_coder</span><span class="p">(</span><span class="nb">object</span><span class="p">)</span></div>
<span class="n">urns</span><span class="o">.</span><span class="n">RunnerApiFn</span><span class="o">.</span><span class="n">register_pickle_urn</span><span class="p">(</span><span class="n">python_urns</span><span class="o">.</span><span class="n">PICKLED_COMBINE_FN</span><span class="p">)</span></div>
<span class="k">class</span> <span class="nc">_ReiterableChain</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Like itertools.chain, but allowing re-iteration.&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">iterables</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">iterables</span> <span class="o">=</span> <span class="n">iterables</span>
<span class="k">def</span> <span class="fm">__iter__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">for</span> <span class="n">iterable</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">iterables</span><span class="p">:</span>
<span class="k">for</span> <span class="n">item</span> <span class="ow">in</span> <span class="n">iterable</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">item</span>
<span class="k">def</span> <span class="fm">__bool__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">for</span> <span class="n">iterable</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">iterables</span><span class="p">:</span>
<span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="n">iterable</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">class</span> <span class="nc">CallableWrapperCombineFn</span><span class="p">(</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> A CombineFn (function) object wrapping a callable object.</span>
<span class="sd"> The purpose of this class is to conveniently wrap simple functions and use</span>
<span class="sd"> them in Combine transforms.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">_DEFAULT_BUFFER_SIZE</span> <span class="o">=</span> <span class="mi">10</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">buffer_size</span><span class="o">=</span><span class="n">_DEFAULT_BUFFER_SIZE</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Initializes a CallableFn object wrapping a callable.</span>
<span class="sd"> Args:</span>
<span class="sd"> fn: A callable object that reduces elements of an iterable to a single</span>
<span class="sd"> value (like the builtins sum and max). This callable must be capable of</span>
<span class="sd"> receiving the kind of values it generates as output in its input, and</span>
<span class="sd"> for best results, its operation must be commutative and associative.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: if fn parameter is not a callable type.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Expected a callable object instead of: </span><span class="si">%r</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">fn</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">_fn</span> <span class="o">=</span> <span class="n">fn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_buffer_size</span> <span class="o">=</span> <span class="n">buffer_size</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="k">return</span> <span class="p">{</span><span class="s1">&#39;fn_dd&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">}</span>
<span class="k">def</span> <span class="fm">__repr__</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">)&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">_fn</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="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="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">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">accumulator</span><span class="p">)</span> <span class="o">&gt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_buffer_size</span><span class="p">:</span>
<span class="n">accumulator</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</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="n">accumulator</span>
<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="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">accumulator</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">elements</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">accumulator</span><span class="p">)</span> <span class="o">&gt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_buffer_size</span><span class="p">:</span>
<span class="n">accumulator</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</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="n">accumulator</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="k">return</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">(</span><span class="n">_ReiterableChain</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="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">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">accumulator</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">return</span> <span class="n">accumulator</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</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">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="bp">self</span><span class="o">.</span><span class="n">_fn</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">def</span> <span class="nf">default_type_hints</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">fn_type_hints</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">)</span>
<span class="n">type_hints</span> <span class="o">=</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">)</span><span class="o">.</span><span class="n">with_defaults</span><span class="p">(</span><span class="n">fn_type_hints</span><span class="p">)</span>
<span class="k">if</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="n">type_hints</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># fn(Iterable[V]) -&gt; V becomes CombineFn(V) -&gt; V</span>
<span class="n">input_args</span><span class="p">,</span> <span class="n">input_kwargs</span> <span class="o">=</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">input_args</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">input_kwargs</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">input_args</span><span class="p">,</span> <span class="n">input_kwargs</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">input_kwargs</span><span class="o">.</span><span class="n">values</span><span class="p">()),</span> <span class="p">{}</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Combiner input type must be specified positionally.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_consistent_with</span><span class="p">(</span><span class="n">input_args</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">]):</span>
<span class="k">raise</span> <span class="n">TypeCheckError</span><span class="p">(</span>
<span class="s1">&#39;All functions for a Combine PTransform must accept a &#39;</span>
<span class="s1">&#39;single argument compatible with: Iterable[Any]. &#39;</span>
<span class="s1">&#39;Instead a function with input type: </span><span class="si">%s</span><span class="s1"> was received.&#39;</span> <span class="o">%</span>
<span class="n">input_args</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span>
<span class="n">input_args</span> <span class="o">=</span> <span class="p">(</span><span class="n">element_type</span><span class="p">(</span><span class="n">input_args</span><span class="p">[</span><span class="mi">0</span><span class="p">]),</span> <span class="p">)</span> <span class="o">+</span> <span class="n">input_args</span><span class="p">[</span><span class="mi">1</span><span class="p">:]</span>
<span class="c1"># TODO(robertwb): Assert output type is consistent with input type?</span>
<span class="k">return</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="o">*</span><span class="n">input_args</span><span class="p">,</span> <span class="o">**</span><span class="n">input_kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">infer_output_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">return</span> <span class="n">_strip_output_annotations</span><span class="p">(</span>
<span class="n">trivial_inference</span><span class="o">.</span><span class="n">infer_return_type</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">,</span> <span class="p">[</span><span class="n">input_type</span><span class="p">]))</span>
<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="c1"># Avoid circular imports.</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="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fn</span> <span class="ow">is</span> <span class="nb">any</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">AnyCombineFn</span><span class="p">()</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fn</span> <span class="ow">is</span> <span class="nb">all</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">AllCombineFn</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">known_types</span> <span class="o">=</span> <span class="p">{</span>
<span class="p">(</span><span class="nb">sum</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">SumInt64Fn</span><span class="p">(),</span>
<span class="p">(</span><span class="nb">min</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">MinInt64Fn</span><span class="p">(),</span>
<span class="p">(</span><span class="nb">max</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">MaxInt64Fn</span><span class="p">(),</span>
<span class="p">(</span><span class="nb">sum</span><span class="p">,</span> <span class="nb">float</span><span class="p">):</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">SumFloatFn</span><span class="p">(),</span>
<span class="p">(</span><span class="nb">min</span><span class="p">,</span> <span class="nb">float</span><span class="p">):</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">MinFloatFn</span><span class="p">(),</span>
<span class="p">(</span><span class="nb">max</span><span class="p">,</span> <span class="nb">float</span><span class="p">):</span> <span class="n">cy_combiners</span><span class="o">.</span><span class="n">MaxFloatFn</span><span class="p">(),</span>
<span class="p">}</span>
<span class="k">return</span> <span class="n">known_types</span><span class="o">.</span><span class="n">get</span><span class="p">((</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">,</span> <span class="n">input_type</span><span class="p">),</span> <span class="bp">self</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">NoSideInputsCallableWrapperCombineFn</span><span class="p">(</span><span class="n">CallableWrapperCombineFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> A CombineFn (function) object wrapping a callable object with no side inputs.</span>
<span class="sd"> This is identical to its parent, but avoids accepting and passing *args</span>
<span class="sd"> and **kwargs for efficiency as they are known to be empty.</span>
<span class="sd"> &quot;&quot;&quot;</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="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="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">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">accumulator</span><span class="p">)</span> <span class="o">&gt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_buffer_size</span><span class="p">:</span>
<span class="n">accumulator</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</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="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="n">accumulator</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">elements</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">accumulator</span><span class="p">)</span> <span class="o">&gt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_buffer_size</span><span class="p">:</span>
<span class="n">accumulator</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</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="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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">(</span><span class="n">_ReiterableChain</span><span class="p">(</span><span class="n">accumulators</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">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">accumulator</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">return</span> <span class="n">accumulator</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">(</span><span class="n">accumulator</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="p">(</span><span class="n">accumulator</span><span class="p">)</span>
<div class="viewcode-block" id="PartitionFn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.PartitionFn">[docs]</a><span class="k">class</span> <span class="nc">PartitionFn</span><span class="p">(</span><span class="n">WithTypeHints</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A function object used by a Partition transform.</span>
<span class="sd"> A PartitionFn specifies how individual values in a PCollection will be placed</span>
<span class="sd"> into separate partitions, indexed by an integer.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="PartitionFn.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.PartitionFn.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="bp">self</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span></div>
<div class="viewcode-block" id="PartitionFn.partition_for"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.PartitionFn.partition_for">[docs]</a> <span class="k">def</span> <span class="nf">partition_for</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">num_partitions</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"># type: (T, int, *typing.Any, **typing.Any) -&gt; int</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Specify which partition will receive this element.</span>
<span class="sd"> Args:</span>
<span class="sd"> element: An element of the input PCollection.</span>
<span class="sd"> num_partitions: Number of partitions, i.e., output PCollections.</span>
<span class="sd"> *args: optional parameters and side inputs.</span>
<span class="sd"> **kwargs: optional parameters and side inputs.</span>
<span class="sd"> Returns:</span>
<span class="sd"> An integer in [0, num_partitions).</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div></div>
<span class="k">class</span> <span class="nc">CallableWrapperPartitionFn</span><span class="p">(</span><span class="n">PartitionFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> A PartitionFn object wrapping a callable object.</span>
<span class="sd"> Instances of this class wrap simple functions for use in Partition operations.</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">fn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Initializes a PartitionFn object wrapping a callable.</span>
<span class="sd"> Args:</span>
<span class="sd"> fn: A callable object, which should accept the following arguments:</span>
<span class="sd"> element - element to assign to a partition.</span>
<span class="sd"> num_partitions - number of output partitions.</span>
<span class="sd"> and may accept additional arguments and side inputs.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: if fn is not a callable type.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Expected a callable object instead of: </span><span class="si">%r</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">fn</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="k">def</span> <span class="nf">partition_for</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">num_partitions</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"># type: (T, int, *typing.Any, **typing.Any) -&gt; int</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">num_partitions</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">_get_function_body_without_inners</span><span class="p">(</span><span class="n">func</span><span class="p">):</span>
<span class="n">source_lines</span> <span class="o">=</span> <span class="n">inspect</span><span class="o">.</span><span class="n">getsourcelines</span><span class="p">(</span><span class="n">func</span><span class="p">)[</span><span class="mi">0</span><span class="p">]</span>
<span class="n">source_lines</span> <span class="o">=</span> <span class="n">dropwhile</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="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s2">&quot;@&quot;</span><span class="p">),</span> <span class="n">source_lines</span><span class="p">)</span>
<span class="n">def_line</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">source_lines</span><span class="p">)</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span>
<span class="k">if</span> <span class="n">def_line</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s2">&quot;def &quot;</span><span class="p">)</span> <span class="ow">and</span> <span class="n">def_line</span><span class="o">.</span><span class="n">endswith</span><span class="p">(</span><span class="s2">&quot;:&quot;</span><span class="p">):</span>
<span class="n">first_line</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">source_lines</span><span class="p">)</span>
<span class="n">indentation</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">first_line</span><span class="p">)</span> <span class="o">-</span> <span class="nb">len</span><span class="p">(</span><span class="n">first_line</span><span class="o">.</span><span class="n">lstrip</span><span class="p">())</span>
<span class="n">final_lines</span> <span class="o">=</span> <span class="p">[</span><span class="n">first_line</span><span class="p">[</span><span class="n">indentation</span><span class="p">:]]</span>
<span class="n">skip_inner_def</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">if</span> <span class="n">first_line</span><span class="p">[</span><span class="n">indentation</span><span class="p">:]</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s2">&quot;def &quot;</span><span class="p">):</span>
<span class="n">skip_inner_def</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">for</span> <span class="n">line</span> <span class="ow">in</span> <span class="n">source_lines</span><span class="p">:</span>
<span class="n">line_indentation</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">line</span><span class="p">)</span> <span class="o">-</span> <span class="nb">len</span><span class="p">(</span><span class="n">line</span><span class="o">.</span><span class="n">lstrip</span><span class="p">())</span>
<span class="k">if</span> <span class="n">line</span><span class="p">[</span><span class="n">indentation</span><span class="p">:]</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s2">&quot;def &quot;</span><span class="p">):</span>
<span class="n">skip_inner_def</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">continue</span>
<span class="k">if</span> <span class="n">skip_inner_def</span> <span class="ow">and</span> <span class="n">line_indentation</span> <span class="o">==</span> <span class="n">indentation</span><span class="p">:</span>
<span class="n">skip_inner_def</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">if</span> <span class="n">skip_inner_def</span> <span class="ow">and</span> <span class="n">line_indentation</span> <span class="o">&gt;</span> <span class="n">indentation</span><span class="p">:</span>
<span class="k">continue</span>
<span class="n">final_lines</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">line</span><span class="p">[</span><span class="n">indentation</span><span class="p">:])</span>
<span class="k">return</span> <span class="s2">&quot;&quot;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">final_lines</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">def_line</span><span class="o">.</span><span class="n">rsplit</span><span class="p">(</span><span class="s2">&quot;:&quot;</span><span class="p">)[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_check_fn_use_yield_and_return</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">types</span><span class="o">.</span><span class="n">BuiltinFunctionType</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">source_code</span> <span class="o">=</span> <span class="n">_get_function_body_without_inners</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="n">has_yield</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">has_return</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">for</span> <span class="n">line</span> <span class="ow">in</span> <span class="n">source_code</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s2">&quot;</span><span class="se">\n</span><span class="s2">&quot;</span><span class="p">):</span>
<span class="k">if</span> <span class="n">line</span><span class="o">.</span><span class="n">lstrip</span><span class="p">()</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s2">&quot;yield &quot;</span><span class="p">)</span> <span class="ow">or</span> <span class="n">line</span><span class="o">.</span><span class="n">lstrip</span><span class="p">()</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span>
<span class="s2">&quot;yield(&quot;</span><span class="p">):</span>
<span class="n">has_yield</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">if</span> <span class="n">line</span><span class="o">.</span><span class="n">lstrip</span><span class="p">()</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s2">&quot;return &quot;</span><span class="p">)</span> <span class="ow">or</span> <span class="n">line</span><span class="o">.</span><span class="n">lstrip</span><span class="p">()</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span>
<span class="s2">&quot;return(&quot;</span><span class="p">):</span>
<span class="n">has_return</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">if</span> <span class="n">has_yield</span> <span class="ow">and</span> <span class="n">has_return</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">except</span> <span class="ne">Exception</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">e</span><span class="p">))</span>
<span class="k">return</span> <span class="kc">False</span>
<div class="viewcode-block" id="ParDo"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo">[docs]</a><span class="k">class</span> <span class="nc">ParDo</span><span class="p">(</span><span class="n">PTransformWithSideInputs</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A :class:`ParDo` transform.</span>
<span class="sd"> Processes an input :class:`~apache_beam.pvalue.PCollection` by applying a</span>
<span class="sd"> :class:`DoFn` to each element and returning the accumulated results into an</span>
<span class="sd"> output :class:`~apache_beam.pvalue.PCollection`. The type of the elements is</span>
<span class="sd"> not fixed as long as the :class:`DoFn` can deal with it. In reality the type</span>
<span class="sd"> is restrained to some extent because the elements sometimes must be persisted</span>
<span class="sd"> to external storage. See the :meth:`.expand()` method comments for a</span>
<span class="sd"> detailed description of all possible arguments.</span>
<span class="sd"> Note that the :class:`DoFn` must return an iterable for each element of the</span>
<span class="sd"> input :class:`~apache_beam.pvalue.PCollection`. An easy way to do this is to</span>
<span class="sd"> use the ``yield`` keyword in the process method.</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll (~apache_beam.pvalue.PCollection):</span>
<span class="sd"> a :class:`~apache_beam.pvalue.PCollection` to be processed.</span>
<span class="sd"> fn (`typing.Union[DoFn, typing.Callable]`): a :class:`DoFn` object to be</span>
<span class="sd"> applied to each element of **pcoll** argument, or a Callable.</span>
<span class="sd"> *args: positional arguments passed to the :class:`DoFn` object.</span>
<span class="sd"> **kwargs: keyword arguments passed to the :class:`DoFn` object.</span>
<span class="sd"> Note that the positional and keyword arguments will be processed in order</span>
<span class="sd"> to detect :class:`~apache_beam.pvalue.PCollection` s that will be computed as</span>
<span class="sd"> side inputs to the transform. During pipeline execution whenever the</span>
<span class="sd"> :class:`DoFn` object gets executed (its :meth:`DoFn.process()` method gets</span>
<span class="sd"> called) the :class:`~apache_beam.pvalue.PCollection` arguments will be</span>
<span class="sd"> replaced by values from the :class:`~apache_beam.pvalue.PCollection` in the</span>
<span class="sd"> exact positions where they appear in the argument lists.</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">fn</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="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="n">fn</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"># TODO(robertwb): Change all uses of the dofn attribute to use fn instead.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">dofn</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">output_tags</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> <span class="c1"># type: typing.Set[str]</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="n">DoFn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;ParDo must be called with a DoFn instance.&#39;</span><span class="p">)</span>
<span class="c1"># DoFn.process cannot allow both return and yield</span>
<span class="k">if</span> <span class="n">_check_fn_use_yield_and_return</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">process</span><span class="p">):</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;Using yield and return in the process method &#39;</span>
<span class="s1">&#39;of </span><span class="si">%s</span><span class="s1"> can lead to unexpected behavior, see:&#39;</span>
<span class="s1">&#39;https://github.com/apache/beam/issues/22969.&#39;</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="vm">__class__</span><span class="p">)</span>
<span class="c1"># Validate the DoFn by creating a DoFnSignature</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.common</span> <span class="kn">import</span> <span class="n">DoFnSignature</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_signature</span> <span class="o">=</span> <span class="n">DoFnSignature</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">)</span>
<div class="viewcode-block" id="ParDo.with_exception_handling"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.with_exception_handling">[docs]</a> <span class="k">def</span> <span class="nf">with_exception_handling</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">main_tag</span><span class="o">=</span><span class="s1">&#39;good&#39;</span><span class="p">,</span>
<span class="n">dead_letter_tag</span><span class="o">=</span><span class="s1">&#39;bad&#39;</span><span class="p">,</span>
<span class="o">*</span><span class="p">,</span>
<span class="n">exc_class</span><span class="o">=</span><span class="ne">Exception</span><span class="p">,</span>
<span class="n">partial</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span>
<span class="n">use_subprocess</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span>
<span class="n">threshold</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span>
<span class="n">threshold_windowing</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Automatically provides a dead letter output for skipping bad records.</span>
<span class="sd"> This can allow a pipeline to continue successfully rather than fail or</span>
<span class="sd"> continuously throw errors on retry when bad elements are encountered.</span>
<span class="sd"> This returns a tagged output with two PCollections, the first being the</span>
<span class="sd"> results of successfully processing the input PCollection, and the second</span>
<span class="sd"> being the set of bad records (those which threw exceptions during</span>
<span class="sd"> processing) along with information about the errors raised.</span>
<span class="sd"> For example, one would write::</span>
<span class="sd"> good, bad = Map(maybe_error_raising_function).with_exception_handling()</span>
<span class="sd"> and `good` will be a PCollection of mapped records and `bad` will contain</span>
<span class="sd"> those that raised exceptions.</span>
<span class="sd"> Args:</span>
<span class="sd"> main_tag: tag to be used for the main (good) output of the DoFn,</span>
<span class="sd"> useful to avoid possible conflicts if this DoFn already produces</span>
<span class="sd"> multiple outputs. Optional, defaults to &#39;good&#39;.</span>
<span class="sd"> dead_letter_tag: tag to be used for the bad records, useful to avoid</span>
<span class="sd"> possible conflicts if this DoFn already produces multiple outputs.</span>
<span class="sd"> Optional, defaults to &#39;bad&#39;.</span>
<span class="sd"> exc_class: An exception class, or tuple of exception classes, to catch.</span>
<span class="sd"> Optional, defaults to &#39;Exception&#39;.</span>
<span class="sd"> partial: Whether to emit outputs for an element as they&#39;re produced</span>
<span class="sd"> (which could result in partial outputs for a ParDo or FlatMap that</span>
<span class="sd"> throws an error part way through execution) or buffer all outputs</span>
<span class="sd"> until successful processing of the entire element. Optional,</span>
<span class="sd"> defaults to False.</span>
<span class="sd"> use_subprocess: Whether to execute the DoFn logic in a subprocess. This</span>
<span class="sd"> allows one to recover from errors that can crash the calling process</span>
<span class="sd"> (e.g. from an underlying C/C++ library causing a segfault), but is</span>
<span class="sd"> slower as elements and results must cross a process boundary. Note</span>
<span class="sd"> that this starts up a long-running process that is used to handle</span>
<span class="sd"> all the elements (until hard failure, which should be rare) rather</span>
<span class="sd"> than a new process per element, so the overhead should be minimal</span>
<span class="sd"> (and can be amortized if there&#39;s any per-process or per-bundle</span>
<span class="sd"> initialization that needs to be done). Optional, defaults to False.</span>
<span class="sd"> threshold: An upper bound on the ratio of records that can be bad before</span>
<span class="sd"> aborting the entire pipeline. Optional, defaults to 1.0 (meaning</span>
<span class="sd"> up to 100% of records can be bad and the pipeline will still succeed).</span>
<span class="sd"> threshold_windowing: Event-time windowing to use for threshold. Optional,</span>
<span class="sd"> defaults to the windowing of the input.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">raw_side_inputs</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">&gt;&gt;</span> <span class="n">_ExceptionHandlingWrapper</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</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="n">main_tag</span><span class="p">,</span>
<span class="n">dead_letter_tag</span><span class="p">,</span>
<span class="n">exc_class</span><span class="p">,</span>
<span class="n">partial</span><span class="p">,</span>
<span class="n">use_subprocess</span><span class="p">,</span>
<span class="n">threshold</span><span class="p">,</span>
<span class="n">threshold_windowing</span><span class="p">)</span></div>
<div class="viewcode-block" id="ParDo.default_type_hints"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.default_type_hints">[docs]</a> <span class="k">def</span> <span class="nf">default_type_hints</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">get_type_hints</span><span class="p">()</span></div>
<div class="viewcode-block" id="ParDo.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_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">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">infer_output_type</span><span class="p">(</span><span class="n">input_type</span><span class="p">)</span></div>
<div class="viewcode-block" id="ParDo.infer_batch_converters"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.infer_batch_converters">[docs]</a> <span class="k">def</span> <span class="nf">infer_batch_converters</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">input_element_type</span><span class="p">):</span>
<span class="c1"># TODO: Test this code (in batch_dofn_test)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">_process_batch_defined</span><span class="p">:</span>
<span class="n">input_batch_type</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">_get_input_batch_type_normalized</span><span class="p">(</span>
<span class="n">input_element_type</span><span class="p">)</span>
<span class="k">if</span> <span class="n">input_batch_type</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;process_batch method on </span><span class="si">{self.fn!r}</span><span class="s2"> does not have &quot;</span>
<span class="s2">&quot;an input type annoation&quot;</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># Generate a batch converter to convert between the input type and the</span>
<span class="c1"># (batch) input type of process_batch</span>
<span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">input_batch_converter</span> <span class="o">=</span> <span class="n">BatchConverter</span><span class="o">.</span><span class="n">from_typehints</span><span class="p">(</span>
<span class="n">element_type</span><span class="o">=</span><span class="n">input_element_type</span><span class="p">,</span> <span class="n">batch_type</span><span class="o">=</span><span class="n">input_batch_type</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">TypeError</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Failed to find a BatchConverter for the input types of DoFn &quot;</span>
<span class="sa">f</span><span class="s2">&quot;</span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="si">!r}</span><span class="s2"> (element_type=</span><span class="si">{</span><span class="n">input_element_type</span><span class="si">!r}</span><span class="s2">, &quot;</span>
<span class="sa">f</span><span class="s2">&quot;batch_type=</span><span class="si">{</span><span class="n">input_batch_type</span><span class="si">!r}</span><span class="s2">).&quot;</span><span class="p">)</span> <span class="kn">from</span> <span class="nn">e</span>
<span class="k">else</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">input_batch_converter</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">_can_yield_batches</span><span class="p">:</span>
<span class="n">output_batch_type</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">_get_output_batch_type_normalized</span><span class="p">(</span>
<span class="n">input_element_type</span><span class="p">)</span>
<span class="k">if</span> <span class="n">output_batch_type</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="c1"># TODO: Mention process method in this error</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="sa">f</span><span class="s2">&quot;process_batch method on </span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="si">!r}</span><span class="s2"> does not have &quot;</span>
<span class="s2">&quot;a return type annoation&quot;</span><span class="p">)</span>
<span class="c1"># Generate a batch converter to convert between the output type and the</span>
<span class="c1"># (batch) output type of process_batch</span>
<span class="n">output_element_type</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">infer_output_type</span><span class="p">(</span><span class="n">input_element_type</span><span class="p">)</span>
<span class="k">try</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">output_batch_converter</span> <span class="o">=</span> <span class="n">BatchConverter</span><span class="o">.</span><span class="n">from_typehints</span><span class="p">(</span>
<span class="n">element_type</span><span class="o">=</span><span class="n">output_element_type</span><span class="p">,</span> <span class="n">batch_type</span><span class="o">=</span><span class="n">output_batch_type</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">TypeError</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Failed to find a BatchConverter for the *output* types of DoFn &quot;</span>
<span class="sa">f</span><span class="s2">&quot;</span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="si">!r}</span><span class="s2"> (element_type=</span><span class="si">{</span><span class="n">output_element_type</span><span class="si">!r}</span><span class="s2">, &quot;</span>
<span class="sa">f</span><span class="s2">&quot;batch_type=</span><span class="si">{</span><span class="n">output_batch_type</span><span class="si">!r}</span><span class="s2">). Maybe you need to override &quot;</span>
<span class="s2">&quot;DoFn.infer_output_type to set the output element type?&quot;</span><span class="p">)</span> <span class="kn">from</span> <span class="nn">e</span>
<span class="k">else</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">output_batch_converter</span> <span class="o">=</span> <span class="kc">None</span></div>
<div class="viewcode-block" id="ParDo.make_fn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.make_fn">[docs]</a> <span class="k">def</span> <span class="nf">make_fn</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">has_side_inputs</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">DoFn</span><span class="p">):</span>
<span class="k">return</span> <span class="n">fn</span>
<span class="k">return</span> <span class="n">CallableWrapperDoFn</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_process_argspec_fn</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">_process_argspec_fn</span><span class="p">()</span>
<div class="viewcode-block" id="ParDo.display_data"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.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;fn&#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">fn</span><span class="o">.</span><span class="vm">__class__</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Transform Function&#39;</span><span class="p">),</span>
<span class="s1">&#39;fn_dd&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span>
<span class="p">}</span></div>
<div class="viewcode-block" id="ParDo.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.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="c1"># In the case of a stateful DoFn, warn if the key coder is not</span>
<span class="c1"># deterministic.</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_signature</span><span class="o">.</span><span class="n">is_stateful_dofn</span><span class="p">():</span>
<span class="n">kv_type_hint</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span>
<span class="k">if</span> <span class="n">kv_type_hint</span> <span class="ow">and</span> <span class="n">kv_type_hint</span> <span class="o">!=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">:</span>
<span class="n">coder</span> <span class="o">=</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="n">kv_type_hint</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">coder</span><span class="o">.</span><span class="n">is_kv_coder</span><span class="p">():</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Input elements to the transform </span><span class="si">%s</span><span class="s1"> with stateful DoFn must be &#39;</span>
<span class="s1">&#39;key-value pairs.&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="p">)</span>
<span class="n">key_coder</span> <span class="o">=</span> <span class="n">coder</span><span class="o">.</span><span class="n">key_coder</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">key_coder</span> <span class="o">=</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="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">key_coder</span><span class="o">.</span><span class="n">is_deterministic</span><span class="p">():</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;Key coder </span><span class="si">%s</span><span class="s1"> for transform </span><span class="si">%s</span><span class="s1"> with stateful DoFn may not &#39;</span>
<span class="s1">&#39;be deterministic. This may cause incorrect behavior for complex &#39;</span>
<span class="s1">&#39;key types. Consider adding an input type hint for this transform.&#39;</span><span class="p">,</span>
<span class="n">key_coder</span><span class="p">,</span>
<span class="bp">self</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_signature</span><span class="o">.</span><span class="n">is_unbounded_per_element</span><span class="p">():</span>
<span class="n">is_bounded</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">is_bounded</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">is_bounded</span>
<span class="bp">self</span><span class="o">.</span><span class="n">infer_batch_converters</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="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="o">.</span><span class="n">from_</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">is_bounded</span><span class="o">=</span><span class="n">is_bounded</span><span class="p">)</span></div>
<div class="viewcode-block" id="ParDo.with_outputs"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.with_outputs">[docs]</a> <span class="k">def</span> <span class="nf">with_outputs</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">tags</span><span class="p">,</span> <span class="n">main</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">allow_unknown_tags</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns a tagged tuple allowing access to the outputs of a</span>
<span class="sd"> :class:`ParDo`.</span>
<span class="sd"> The resulting object supports access to the</span>
<span class="sd"> :class:`~apache_beam.pvalue.PCollection` associated with a tag</span>
<span class="sd"> (e.g. ``o.tag``, ``o[tag]``) and iterating over the available tags</span>
<span class="sd"> (e.g. ``for tag in o: ...``).</span>
<span class="sd"> Args:</span>
<span class="sd"> *tags: if non-empty, list of valid tags. If a list of valid tags is given,</span>
<span class="sd"> it will be an error to use an undeclared tag later in the pipeline.</span>
<span class="sd"> **main_kw: dictionary empty or with one key ``&#39;main&#39;`` defining the tag to</span>
<span class="sd"> be used for the main output (which will not have a tag associated with</span>
<span class="sd"> it).</span>
<span class="sd"> Returns:</span>
<span class="sd"> ~apache_beam.pvalue.DoOutputsTuple: An object of type</span>
<span class="sd"> :class:`~apache_beam.pvalue.DoOutputsTuple` that bundles together all</span>
<span class="sd"> the outputs of a :class:`ParDo` transform and allows accessing the</span>
<span class="sd"> individual :class:`~apache_beam.pvalue.PCollection` s for each output</span>
<span class="sd"> using an ``object.tag`` syntax.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: if the **self** object is not a</span>
<span class="sd"> :class:`~apache_beam.pvalue.PCollection` that is the result of a</span>
<span class="sd"> :class:`ParDo` transform.</span>
<span class="sd"> ValueError: if **main_kw** contains any key other than</span>
<span class="sd"> ``&#39;main&#39;``.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">main</span> <span class="ow">in</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="s1">&#39;Main output tag </span><span class="si">%r</span><span class="s1"> must be different from side output tags </span><span class="si">%r</span><span class="s1">.&#39;</span> <span class="o">%</span>
<span class="p">(</span><span class="n">main</span><span class="p">,</span> <span class="n">tags</span><span class="p">))</span>
<span class="k">return</span> <span class="n">_MultiParDo</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tags</span><span class="p">,</span> <span class="n">main</span><span class="p">,</span> <span class="n">allow_unknown_tags</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_do_fn_info</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">DoFnInfo</span><span class="o">.</span><span class="n">create</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</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">_get_key_and_window_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">named_inputs</span><span class="p">):</span>
<span class="k">if</span> <span class="n">named_inputs</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_signature</span><span class="o">.</span><span class="n">is_stateful_dofn</span><span class="p">():</span>
<span class="k">return</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">None</span>
<span class="n">main_input</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">named_inputs</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> <span class="o">-</span> <span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">))[</span><span class="mi">0</span><span class="p">]</span>
<span class="n">input_pcoll</span> <span class="o">=</span> <span class="n">named_inputs</span><span class="p">[</span><span class="n">main_input</span><span class="p">]</span>
<span class="n">kv_type_hint</span> <span class="o">=</span> <span class="n">input_pcoll</span><span class="o">.</span><span class="n">element_type</span>
<span class="k">if</span> <span class="n">kv_type_hint</span> <span class="ow">and</span> <span class="n">kv_type_hint</span> <span class="o">!=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">:</span>
<span class="n">coder</span> <span class="o">=</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="n">kv_type_hint</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">coder</span><span class="o">.</span><span class="n">is_kv_coder</span><span class="p">():</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Input elements to the transform </span><span class="si">%s</span><span class="s1"> with stateful DoFn must be &#39;</span>
<span class="s1">&#39;key-value pairs.&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="p">)</span>
<span class="n">key_coder</span> <span class="o">=</span> <span class="n">coder</span><span class="o">.</span><span class="n">key_coder</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">key_coder</span> <span class="o">=</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="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">)</span>
<span class="n">window_coder</span> <span class="o">=</span> <span class="n">input_pcoll</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">return</span> <span class="n">key_coder</span><span class="p">,</span> <span class="n">window_coder</span>
<span class="c1"># typing: PTransform base class does not accept extra_kwargs</span>
<div class="viewcode-block" id="ParDo.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</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="o">**</span><span class="n">extra_kwargs</span><span class="p">):</span> <span class="c1"># type: ignore[override]</span>
<span class="c1"># type: (PipelineContext, **typing.Any) -&gt; typing.Tuple[str, message.Message]</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">ParDo</span><span class="p">),</span> \
<span class="s2">&quot;expected instance of ParDo, but got </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="vm">__class__</span>
<span class="n">state_specs</span><span class="p">,</span> <span class="n">timer_specs</span> <span class="o">=</span> <span class="n">userstate</span><span class="o">.</span><span class="n">get_dofn_specs</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">)</span>
<span class="k">if</span> <span class="n">state_specs</span> <span class="ow">or</span> <span class="n">timer_specs</span><span class="p">:</span>
<span class="n">context</span><span class="o">.</span><span class="n">add_requirement</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">requirements</span><span class="o">.</span><span class="n">REQUIRES_STATEFUL_PROCESSING</span><span class="o">.</span><span class="n">urn</span><span class="p">)</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.common</span> <span class="kn">import</span> <span class="n">DoFnSignature</span>
<span class="n">sig</span> <span class="o">=</span> <span class="n">DoFnSignature</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">)</span>
<span class="n">is_splittable</span> <span class="o">=</span> <span class="n">sig</span><span class="o">.</span><span class="n">is_splittable_dofn</span><span class="p">()</span>
<span class="k">if</span> <span class="n">is_splittable</span><span class="p">:</span>
<span class="n">restriction_coder</span> <span class="o">=</span> <span class="n">sig</span><span class="o">.</span><span class="n">get_restriction_coder</span><span class="p">()</span>
<span class="c1"># restriction_coder will never be None when is_splittable is True</span>
<span class="k">assert</span> <span class="n">restriction_coder</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="n">restriction_coder_id</span> <span class="o">=</span> <span class="n">context</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span>
<span class="n">restriction_coder</span><span class="p">)</span> <span class="c1"># type: typing.Optional[str]</span>
<span class="n">context</span><span class="o">.</span><span class="n">add_requirement</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">requirements</span><span class="o">.</span><span class="n">REQUIRES_SPLITTABLE_DOFN</span><span class="o">.</span><span class="n">urn</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">restriction_coder_id</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">has_bundle_finalization</span> <span class="o">=</span> <span class="n">sig</span><span class="o">.</span><span class="n">has_bundle_finalization</span><span class="p">()</span>
<span class="k">if</span> <span class="n">has_bundle_finalization</span><span class="p">:</span>
<span class="n">context</span><span class="o">.</span><span class="n">add_requirement</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">requirements</span><span class="o">.</span><span class="n">REQUIRES_BUNDLE_FINALIZATION</span><span class="o">.</span><span class="n">urn</span><span class="p">)</span>
<span class="c1"># Get key_coder and window_coder for main_input.</span>
<span class="n">key_coder</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">_get_key_and_window_coder</span><span class="p">(</span>
<span class="n">extra_kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;named_inputs&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">PAR_DO</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ParDoPayload</span><span class="p">(</span>
<span class="n">do_fn</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_do_fn_info</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="n">requests_finalization</span><span class="o">=</span><span class="n">has_bundle_finalization</span><span class="p">,</span>
<span class="n">restriction_coder_id</span><span class="o">=</span><span class="n">restriction_coder_id</span><span class="p">,</span>
<span class="n">state_specs</span><span class="o">=</span><span class="p">{</span>
<span class="n">spec</span><span class="o">.</span><span class="n">name</span><span class="p">:</span> <span class="n">spec</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="k">for</span> <span class="n">spec</span> <span class="ow">in</span> <span class="n">state_specs</span>
<span class="p">},</span>
<span class="n">timer_family_specs</span><span class="o">=</span><span class="p">{</span>
<span class="n">spec</span><span class="o">.</span><span class="n">name</span><span class="p">:</span> <span class="n">spec</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="n">key_coder</span><span class="p">,</span> <span class="n">window_coder</span><span class="p">)</span>
<span class="k">for</span> <span class="n">spec</span> <span class="ow">in</span> <span class="n">timer_specs</span>
<span class="p">},</span>
<span class="c1"># It&#39;d be nice to name these according to their actual</span>
<span class="c1"># names/positions in the orignal argument list, but such a</span>
<span class="c1"># transformation is currently irreversible given how</span>
<span class="c1"># remove_objects_from_args and insert_values_in_args</span>
<span class="c1"># are currently implemented.</span>
<span class="n">side_inputs</span><span class="o">=</span><span class="p">{(</span><span class="n">SIDE_INPUT_PREFIX</span> <span class="o">+</span> <span class="s1">&#39;</span><span class="si">%s</span><span class="s1">&#39;</span><span class="p">)</span> <span class="o">%</span> <span class="n">ix</span><span class="p">:</span>
<span class="n">si</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="k">for</span> <span class="n">ix</span><span class="p">,</span>
<span class="n">si</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">)}))</span></div>
<div class="viewcode-block" id="ParDo.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">PAR_DO</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ParDoPayload</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">pardo_payload</span><span class="p">,</span> <span class="n">context</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="n">si_tags_and_types</span><span class="p">,</span> <span class="n">windowing</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">DoFnInfo</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span>
<span class="n">pardo_payload</span><span class="o">.</span><span class="n">do_fn</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span><span class="o">.</span><span class="n">serialized_dofn_data</span><span class="p">())</span>
<span class="k">if</span> <span class="n">si_tags_and_types</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;explicit side input data&#39;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">windowing</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;explicit windowing&#39;</span><span class="p">)</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">fn</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"># This is an ordered list stored as a dict (see the comments in</span>
<span class="c1"># to_runner_api_parameter above).</span>
<span class="n">indexed_side_inputs</span> <span class="o">=</span> <span class="p">[(</span>
<span class="n">get_sideinput_index</span><span class="p">(</span><span class="n">tag</span><span class="p">),</span>
<span class="n">pvalue</span><span class="o">.</span><span class="n">AsSideInput</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">si</span><span class="p">,</span> <span class="n">context</span><span class="p">))</span> <span class="k">for</span> <span class="n">tag</span><span class="p">,</span>
<span class="n">si</span> <span class="ow">in</span> <span class="n">pardo_payload</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">items</span><span class="p">()]</span>
<span class="n">result</span><span class="o">.</span><span class="n">side_inputs</span> <span class="o">=</span> <span class="p">[</span><span class="n">si</span> <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">si</span> <span class="ow">in</span> <span class="nb">sorted</span><span class="p">(</span><span class="n">indexed_side_inputs</span><span class="p">)]</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="ParDo.runner_api_requires_keyed_input"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.runner_api_requires_keyed_input">[docs]</a> <span class="k">def</span> <span class="nf">runner_api_requires_keyed_input</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">userstate</span><span class="o">.</span><span class="n">is_stateful_dofn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">)</span></div>
<div class="viewcode-block" id="ParDo.get_restriction_coder"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.ParDo.get_restriction_coder">[docs]</a> <span class="k">def</span> <span class="nf">get_restriction_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns `restriction coder if `DoFn` of this `ParDo` is a SDF.</span>
<span class="sd"> Returns `None` otherwise.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.common</span> <span class="kn">import</span> <span class="n">DoFnSignature</span>
<span class="k">return</span> <span class="n">DoFnSignature</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">)</span><span class="o">.</span><span class="n">get_restriction_coder</span><span class="p">()</span></div>
<span class="k">def</span> <span class="nf">_add_type_constraint_from_consumer</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">full_label</span><span class="p">,</span> <span class="n">input_type_hints</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="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="s1">&#39;_runtime_output_constraints&#39;</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">_runtime_output_constraints</span> <span class="o">=</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">_runtime_output_constraints</span><span class="p">[</span><span class="n">full_label</span><span class="p">]</span> <span class="o">=</span> <span class="n">input_type_hints</span></div>
<span class="k">class</span> <span class="nc">_MultiParDo</span><span class="p">(</span><span class="n">PTransform</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">do_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="n">allow_unknown_tags</span><span class="o">=</span><span class="kc">None</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">do_transform</span><span class="o">.</span><span class="n">label</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_do_transform</span> <span class="o">=</span> <span class="n">do_transform</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">_allow_unknown_tags</span> <span class="o">=</span> <span class="n">allow_unknown_tags</span>
<span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="n">_</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="bp">self</span><span class="o">.</span><span class="n">_do_transform</span>
<span class="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">DoOutputsTuple</span><span class="p">(</span>
<span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_do_transform</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">_main_tag</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_allow_unknown_tags</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">DoFnInfo</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;This class represents the state in the ParDoPayload&#39;s function spec,</span>
<span class="sd"> which is the actual DoFn together with some data required for invoking it.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">register_stateless_dofn</span><span class="p">(</span><span class="n">urn</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">wrapper</span><span class="p">(</span><span class="bp">cls</span><span class="p">):</span>
<span class="n">StatelessDoFnInfo</span><span class="o">.</span><span class="n">REGISTERED_DOFNS</span><span class="p">[</span><span class="n">urn</span><span class="p">]</span> <span class="o">=</span> <span class="bp">cls</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_stateless_dofn_urn</span> <span class="o">=</span> <span class="n">urn</span>
<span class="k">return</span> <span class="bp">cls</span>
<span class="k">return</span> <span class="n">wrapper</span>
<span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">create</span><span class="p">(</span><span class="bp">cls</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="nb">hasattr</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="s1">&#39;_stateless_dofn_urn&#39;</span><span class="p">):</span>
<span class="k">assert</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="k">return</span> <span class="n">StatelessDoFnInfo</span><span class="p">(</span><span class="n">fn</span><span class="o">.</span><span class="n">_stateless_dofn_urn</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">PickledDoFnInfo</span><span class="p">(</span><span class="bp">cls</span><span class="o">.</span><span class="n">_pickled_do_fn_info</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="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api</span><span class="p">(</span><span class="n">spec</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">if</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_DOFN_INFO</span><span class="p">:</span>
<span class="k">return</span> <span class="n">PickledDoFnInfo</span><span class="p">(</span><span class="n">spec</span><span class="o">.</span><span class="n">payload</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">spec</span><span class="o">.</span><span class="n">urn</span> <span class="ow">in</span> <span class="n">StatelessDoFnInfo</span><span class="o">.</span><span class="n">REGISTERED_DOFNS</span><span class="p">:</span>
<span class="k">return</span> <span class="n">StatelessDoFnInfo</span><span class="p">(</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span><span class="p">)</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 DoFn type: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">spec</span><span class="o">.</span><span class="n">urn</span><span class="p">)</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_pickled_do_fn_info</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="c1"># This can be cleaned up once all runners move to portability.</span>
<span class="k">return</span> <span class="n">pickler</span><span class="o">.</span><span class="n">dumps</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="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">serialized_dofn_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span>
<span class="k">class</span> <span class="nc">PickledDoFnInfo</span><span class="p">(</span><span class="n">DoFnInfo</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">serialized_data</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_serialized_data</span> <span class="o">=</span> <span class="n">serialized_data</span>
<span class="k">def</span> <span class="nf">serialized_dofn_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">_serialized_data</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">unused_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">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_DOFN_INFO</span><span class="p">,</span> <span class="n">payload</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_serialized_data</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">StatelessDoFnInfo</span><span class="p">(</span><span class="n">DoFnInfo</span><span class="p">):</span>
<span class="n">REGISTERED_DOFNS</span> <span class="o">=</span> <span class="p">{}</span> <span class="c1"># type: typing.Dict[str, typing.Type[DoFn]]</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">urn</span><span class="p">):</span>
<span class="c1"># type: (str) -&gt; None</span>
<span class="k">assert</span> <span class="n">urn</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">REGISTERED_DOFNS</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_urn</span> <span class="o">=</span> <span class="n">urn</span>
<span class="k">def</span> <span class="nf">serialized_dofn_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">_pickled_do_fn_info</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">REGISTERED_DOFNS</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_urn</span><span class="p">](),</span> <span class="p">(),</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">unused_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">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">_urn</span><span class="p">)</span>
<div class="viewcode-block" id="FlatMap"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.FlatMap">[docs]</a><span class="k">def</span> <span class="nf">FlatMap</span><span class="p">(</span><span class="n">fn</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"># pylint: disable=invalid-name</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;:func:`FlatMap` is like :class:`ParDo` except it takes a callable to</span>
<span class="sd"> specify the transformation.</span>
<span class="sd"> The callable must return an iterable for each element of the input</span>
<span class="sd"> :class:`~apache_beam.pvalue.PCollection`. The elements of these iterables will</span>
<span class="sd"> be flattened into the output :class:`~apache_beam.pvalue.PCollection`.</span>
<span class="sd"> Args:</span>
<span class="sd"> fn (callable): a callable object.</span>
<span class="sd"> *args: positional arguments passed to the transform callable.</span>
<span class="sd"> **kwargs: keyword arguments passed to the transform callable.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ~apache_beam.pvalue.PCollection:</span>
<span class="sd"> A :class:`~apache_beam.pvalue.PCollection` containing the</span>
<span class="sd"> :func:`FlatMap` outputs.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: If the **fn** passed as argument is not a callable.</span>
<span class="sd"> Typical error is to pass a :class:`DoFn` instance which is supported only</span>
<span class="sd"> for :class:`ParDo`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">label</span> <span class="o">=</span> <span class="s1">&#39;FlatMap(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;FlatMap can be used only with callable objects. &#39;</span>
<span class="s1">&#39;Received </span><span class="si">%r</span><span class="s1"> instead.&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">fn</span><span class="p">))</span>
<span class="n">pardo</span> <span class="o">=</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">CallableWrapperDoFn</span><span class="p">(</span><span class="n">fn</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">pardo</span><span class="o">.</span><span class="n">label</span> <span class="o">=</span> <span class="n">label</span>
<span class="k">return</span> <span class="n">pardo</span></div>
<div class="viewcode-block" id="Map"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Map">[docs]</a><span class="k">def</span> <span class="nf">Map</span><span class="p">(</span><span class="n">fn</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"># pylint: disable=invalid-name</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;:func:`Map` is like :func:`FlatMap` except its callable returns only a</span>
<span class="sd"> single element.</span>
<span class="sd"> Args:</span>
<span class="sd"> fn (callable): a callable object.</span>
<span class="sd"> *args: positional arguments passed to the transform callable.</span>
<span class="sd"> **kwargs: keyword arguments passed to the transform callable.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ~apache_beam.pvalue.PCollection:</span>
<span class="sd"> A :class:`~apache_beam.pvalue.PCollection` containing the</span>
<span class="sd"> :func:`Map` outputs.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: If the **fn** passed as argument is not a callable.</span>
<span class="sd"> Typical error is to pass a :class:`DoFn` instance which is supported only</span>
<span class="sd"> for :class:`ParDo`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;Map can be used only with callable objects. &#39;</span>
<span class="s1">&#39;Received </span><span class="si">%r</span><span class="s1"> instead.&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">fn</span><span class="p">))</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.util</span> <span class="kn">import</span> <span class="n">fn_takes_side_inputs</span>
<span class="k">if</span> <span class="n">fn_takes_side_inputs</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</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="p">[</span><span class="n">fn</span><span class="p">(</span><span class="n">x</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">else</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">[</span><span class="n">fn</span><span class="p">(</span><span class="n">x</span><span class="p">)]</span>
<span class="n">label</span> <span class="o">=</span> <span class="s1">&#39;Map(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="c1"># TODO. What about callable classes?</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="s1">&#39;__name__&#39;</span><span class="p">):</span>
<span class="n">wrapper</span><span class="o">.</span><span class="vm">__name__</span> <span class="o">=</span> <span class="n">fn</span><span class="o">.</span><span class="vm">__name__</span>
<span class="c1"># Proxy the type-hint information from the original function to this new</span>
<span class="c1"># wrapped function.</span>
<span class="n">type_hints</span> <span class="o">=</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span><span class="o">.</span><span class="n">with_defaults</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">))</span>
<span class="k">if</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="n">with_input_types</span><span class="p">(</span>
<span class="o">*</span><span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="o">**</span><span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span><span class="p">[</span><span class="mi">1</span><span class="p">])(</span>
<span class="n">wrapper</span><span class="p">)</span>
<span class="n">output_hint</span> <span class="o">=</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">simple_output_type</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">output_hint</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="n">with_output_types</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">_strip_output_annotations</span><span class="p">(</span><span class="n">output_hint</span><span class="p">)])(</span>
<span class="n">wrapper</span><span class="p">)</span>
<span class="c1"># pylint: disable=protected-access</span>
<span class="n">wrapper</span><span class="o">.</span><span class="n">_argspec_fn</span> <span class="o">=</span> <span class="n">fn</span>
<span class="c1"># pylint: enable=protected-access</span>
<span class="n">pardo</span> <span class="o">=</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">wrapper</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">pardo</span><span class="o">.</span><span class="n">label</span> <span class="o">=</span> <span class="n">label</span>
<span class="k">return</span> <span class="n">pardo</span></div>
<div class="viewcode-block" id="MapTuple"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.MapTuple">[docs]</a><span class="k">def</span> <span class="nf">MapTuple</span><span class="p">(</span><span class="n">fn</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"># pylint: disable=invalid-name</span>
<span class="w"> </span><span class="sa">r</span><span class="sd">&quot;&quot;&quot;:func:`MapTuple` is like :func:`Map` but expects tuple inputs and</span>
<span class="sd"> flattens them into multiple input arguments.</span>
<span class="sd"> beam.MapTuple(lambda a, b, ...: ...)</span>
<span class="sd"> In other words</span>
<span class="sd"> beam.MapTuple(fn)</span>
<span class="sd"> is equivalent to</span>
<span class="sd"> beam.Map(lambda element, ...: fn(\*element, ...))</span>
<span class="sd"> This can be useful when processing a PCollection of tuples</span>
<span class="sd"> (e.g. key-value pairs).</span>
<span class="sd"> Args:</span>
<span class="sd"> fn (callable): a callable object.</span>
<span class="sd"> *args: positional arguments passed to the transform callable.</span>
<span class="sd"> **kwargs: keyword arguments passed to the transform callable.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ~apache_beam.pvalue.PCollection:</span>
<span class="sd"> A :class:`~apache_beam.pvalue.PCollection` containing the</span>
<span class="sd"> :func:`MapTuple` outputs.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: If the **fn** passed as argument is not a callable.</span>
<span class="sd"> Typical error is to pass a :class:`DoFn` instance which is supported only</span>
<span class="sd"> for :class:`ParDo`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;MapTuple can be used only with callable objects. &#39;</span>
<span class="s1">&#39;Received </span><span class="si">%r</span><span class="s1"> instead.&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">fn</span><span class="p">))</span>
<span class="n">label</span> <span class="o">=</span> <span class="s1">&#39;MapTuple(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="n">arg_names</span><span class="p">,</span> <span class="n">defaults</span> <span class="o">=</span> <span class="n">get_function_args_defaults</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="n">num_defaults</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">defaults</span><span class="p">)</span>
<span class="k">if</span> <span class="n">num_defaults</span> <span class="o">&lt;</span> <span class="nb">len</span><span class="p">(</span><span class="n">args</span><span class="p">)</span> <span class="o">+</span> <span class="nb">len</span><span class="p">(</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Side inputs must have defaults for MapTuple.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">defaults</span> <span class="ow">or</span> <span class="n">args</span> <span class="ow">or</span> <span class="n">kwargs</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</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="p">[</span><span class="n">fn</span><span class="p">(</span><span class="o">*</span><span class="p">(</span><span class="nb">tuple</span><span class="p">(</span><span class="n">x</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">else</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">[</span><span class="n">fn</span><span class="p">(</span><span class="o">*</span><span class="n">x</span><span class="p">)]</span>
<span class="c1"># Proxy the type-hint information from the original function to this new</span>
<span class="c1"># wrapped function.</span>
<span class="n">type_hints</span> <span class="o">=</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span><span class="o">.</span><span class="n">with_defaults</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">))</span>
<span class="k">if</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="c1"># TODO(BEAM-14052): ignore input hints, as we do not have enough</span>
<span class="c1"># information to infer the input type hint of the wrapper function.</span>
<span class="k">pass</span>
<span class="n">output_hint</span> <span class="o">=</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">simple_output_type</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">output_hint</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="n">with_output_types</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">_strip_output_annotations</span><span class="p">(</span><span class="n">output_hint</span><span class="p">)])(</span>
<span class="n">wrapper</span><span class="p">)</span>
<span class="c1"># Replace the first (args) component.</span>
<span class="n">modified_arg_names</span> <span class="o">=</span> <span class="p">[</span><span class="s1">&#39;tuple_element&#39;</span><span class="p">]</span> <span class="o">+</span> <span class="n">arg_names</span><span class="p">[</span><span class="o">-</span><span class="n">num_defaults</span><span class="p">:]</span>
<span class="n">modified_argspec</span> <span class="o">=</span> <span class="p">(</span><span class="n">modified_arg_names</span><span class="p">,</span> <span class="n">defaults</span><span class="p">)</span>
<span class="n">pardo</span> <span class="o">=</span> <span class="n">ParDo</span><span class="p">(</span>
<span class="n">CallableWrapperDoFn</span><span class="p">(</span><span class="n">wrapper</span><span class="p">,</span> <span class="n">fullargspec</span><span class="o">=</span><span class="n">modified_argspec</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">pardo</span><span class="o">.</span><span class="n">label</span> <span class="o">=</span> <span class="n">label</span>
<span class="k">return</span> <span class="n">pardo</span></div>
<div class="viewcode-block" id="FlatMapTuple"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.FlatMapTuple">[docs]</a><span class="k">def</span> <span class="nf">FlatMapTuple</span><span class="p">(</span><span class="n">fn</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"># pylint: disable=invalid-name</span>
<span class="w"> </span><span class="sa">r</span><span class="sd">&quot;&quot;&quot;:func:`FlatMapTuple` is like :func:`FlatMap` but expects tuple inputs and</span>
<span class="sd"> flattens them into multiple input arguments.</span>
<span class="sd"> beam.FlatMapTuple(lambda a, b, ...: ...)</span>
<span class="sd"> is equivalent to Python 2</span>
<span class="sd"> beam.FlatMap(lambda (a, b, ...), ...: ...)</span>
<span class="sd"> In other words</span>
<span class="sd"> beam.FlatMapTuple(fn)</span>
<span class="sd"> is equivalent to</span>
<span class="sd"> beam.FlatMap(lambda element, ...: fn(\*element, ...))</span>
<span class="sd"> This can be useful when processing a PCollection of tuples</span>
<span class="sd"> (e.g. key-value pairs).</span>
<span class="sd"> Args:</span>
<span class="sd"> fn (callable): a callable object.</span>
<span class="sd"> *args: positional arguments passed to the transform callable.</span>
<span class="sd"> **kwargs: keyword arguments passed to the transform callable.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ~apache_beam.pvalue.PCollection:</span>
<span class="sd"> A :class:`~apache_beam.pvalue.PCollection` containing the</span>
<span class="sd"> :func:`FlatMapTuple` outputs.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: If the **fn** passed as argument is not a callable.</span>
<span class="sd"> Typical error is to pass a :class:`DoFn` instance which is supported only</span>
<span class="sd"> for :class:`ParDo`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;FlatMapTuple can be used only with callable objects. &#39;</span>
<span class="s1">&#39;Received </span><span class="si">%r</span><span class="s1"> instead.&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">fn</span><span class="p">))</span>
<span class="n">label</span> <span class="o">=</span> <span class="s1">&#39;FlatMapTuple(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="n">arg_names</span><span class="p">,</span> <span class="n">defaults</span> <span class="o">=</span> <span class="n">get_function_args_defaults</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="n">num_defaults</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">defaults</span><span class="p">)</span>
<span class="k">if</span> <span class="n">num_defaults</span> <span class="o">&lt;</span> <span class="nb">len</span><span class="p">(</span><span class="n">args</span><span class="p">)</span> <span class="o">+</span> <span class="nb">len</span><span class="p">(</span><span class="n">kwargs</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">&#39;Side inputs must have defaults for FlatMapTuple.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">defaults</span> <span class="ow">or</span> <span class="n">args</span> <span class="ow">or</span> <span class="n">kwargs</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</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">fn</span><span class="p">(</span><span class="o">*</span><span class="p">(</span><span class="nb">tuple</span><span class="p">(</span><span class="n">x</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">else</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">fn</span><span class="p">(</span><span class="o">*</span><span class="n">x</span><span class="p">)</span>
<span class="c1"># Proxy the type-hint information from the original function to this new</span>
<span class="c1"># wrapped function.</span>
<span class="n">type_hints</span> <span class="o">=</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span><span class="o">.</span><span class="n">with_defaults</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">))</span>
<span class="k">if</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="c1"># TODO(BEAM-14052): ignore input hints, as we do not have enough</span>
<span class="c1"># information to infer the input type hint of the wrapper function.</span>
<span class="k">pass</span>
<span class="n">output_hint</span> <span class="o">=</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">simple_output_type</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">output_hint</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="n">with_output_types</span><span class="p">(</span><span class="n">_strip_output_annotations</span><span class="p">(</span><span class="n">output_hint</span><span class="p">))(</span><span class="n">wrapper</span><span class="p">)</span>
<span class="c1"># Replace the first (args) component.</span>
<span class="n">modified_arg_names</span> <span class="o">=</span> <span class="p">[</span><span class="s1">&#39;tuple_element&#39;</span><span class="p">]</span> <span class="o">+</span> <span class="n">arg_names</span><span class="p">[</span><span class="o">-</span><span class="n">num_defaults</span><span class="p">:]</span>
<span class="n">modified_argspec</span> <span class="o">=</span> <span class="p">(</span><span class="n">modified_arg_names</span><span class="p">,</span> <span class="n">defaults</span><span class="p">)</span>
<span class="n">pardo</span> <span class="o">=</span> <span class="n">ParDo</span><span class="p">(</span>
<span class="n">CallableWrapperDoFn</span><span class="p">(</span><span class="n">wrapper</span><span class="p">,</span> <span class="n">fullargspec</span><span class="o">=</span><span class="n">modified_argspec</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">pardo</span><span class="o">.</span><span class="n">label</span> <span class="o">=</span> <span class="n">label</span>
<span class="k">return</span> <span class="n">pardo</span></div>
<span class="k">class</span> <span class="nc">_ExceptionHandlingWrapper</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;Implementation of ParDo.with_exception_handling.&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="n">main_tag</span><span class="p">,</span>
<span class="n">dead_letter_tag</span><span class="p">,</span>
<span class="n">exc_class</span><span class="p">,</span>
<span class="n">partial</span><span class="p">,</span>
<span class="n">use_subprocess</span><span class="p">,</span>
<span class="n">threshold</span><span class="p">,</span>
<span class="n">threshold_windowing</span><span class="p">):</span>
<span class="k">if</span> <span class="n">partial</span> <span class="ow">and</span> <span class="n">use_subprocess</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;partial and use_subprocess are mutually incompatible.&#39;</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="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">_dead_letter_tag</span> <span class="o">=</span> <span class="n">dead_letter_tag</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_exc_class</span> <span class="o">=</span> <span class="n">exc_class</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_partial</span> <span class="o">=</span> <span class="n">partial</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_use_subprocess</span> <span class="o">=</span> <span class="n">use_subprocess</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_threshold</span> <span class="o">=</span> <span class="n">threshold</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_threshold_windowing</span> <span class="o">=</span> <span class="n">threshold_windowing</span>
<span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span>
<span class="n">_ExceptionHandlingWrapperDoFn</span><span class="p">(</span>
<span class="n">_SubprocessDoFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">)</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_use_subprocess</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_dead_letter_tag</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_exc_class</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_partial</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="o">.</span><span class="n">with_outputs</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_dead_letter_tag</span><span class="p">,</span> <span class="n">main</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">allow_unknown_tags</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_threshold</span> <span class="o">&lt;</span> <span class="mf">1.0</span><span class="p">:</span>
<span class="k">class</span> <span class="nc">MaybeWindow</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="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">expand</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">_threshold_windowing</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">WindowInto</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_threshold_windowing</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="n">input_count_view</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">&#39;CountTotal&#39;</span> <span class="o">&gt;&gt;</span> <span class="p">(</span>
<span class="n">MaybeWindow</span><span class="p">()</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">_</span><span class="p">:</span> <span class="mi">1</span><span class="p">)</span>
<span class="o">|</span> <span class="n">CombineGlobally</span><span class="p">(</span><span class="nb">sum</span><span class="p">)</span><span class="o">.</span><span class="n">as_singleton_view</span><span class="p">())</span>
<span class="n">bad_count_pcoll</span> <span class="o">=</span> <span class="n">result</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_dead_letter_tag</span><span class="p">]</span> <span class="o">|</span> <span class="s1">&#39;CountBad&#39;</span> <span class="o">&gt;&gt;</span> <span class="p">(</span>
<span class="n">MaybeWindow</span><span class="p">()</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">_</span><span class="p">:</span> <span class="mi">1</span><span class="p">)</span>
<span class="o">|</span> <span class="n">CombineGlobally</span><span class="p">(</span><span class="nb">sum</span><span class="p">)</span><span class="o">.</span><span class="n">without_defaults</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">check_threshold</span><span class="p">(</span><span class="n">bad</span><span class="p">,</span> <span class="n">total</span><span class="p">,</span> <span class="n">threshold</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">):</span>
<span class="k">if</span> <span class="n">bad</span> <span class="o">&gt;</span> <span class="n">total</span> <span class="o">*</span> <span class="n">threshold</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;The number of failing elements within the window </span><span class="si">%r</span><span class="s1"> &#39;</span>
<span class="s1">&#39;exceeded threshold: </span><span class="si">%s</span><span class="s1"> / </span><span class="si">%s</span><span class="s1"> = </span><span class="si">%s</span><span class="s1"> &gt; </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span>
<span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">bad</span><span class="p">,</span> <span class="n">total</span><span class="p">,</span> <span class="n">bad</span> <span class="o">/</span> <span class="n">total</span><span class="p">,</span> <span class="n">threshold</span><span class="p">))</span>
<span class="n">_</span> <span class="o">=</span> <span class="n">bad_count_pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span>
<span class="n">check_threshold</span><span class="p">,</span> <span class="n">input_count_view</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_threshold</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span>
<span class="k">class</span> <span class="nc">_ExceptionHandlingWrapperDoFn</span><span class="p">(</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">fn</span><span class="p">,</span> <span class="n">dead_letter_tag</span><span class="p">,</span> <span class="n">exc_class</span><span class="p">,</span> <span class="n">partial</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">_dead_letter_tag</span> <span class="o">=</span> <span class="n">dead_letter_tag</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_exc_class</span> <span class="o">=</span> <span class="n">exc_class</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_partial</span> <span class="o">=</span> <span class="n">partial</span>
<span class="k">def</span> <span class="fm">__getattribute__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">):</span>
<span class="k">if</span> <span class="p">(</span><span class="n">name</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">&#39;__&#39;</span><span class="p">)</span> <span class="ow">or</span> <span class="n">name</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="vm">__dict__</span> <span class="ow">or</span>
<span class="n">name</span> <span class="ow">in</span> <span class="n">_ExceptionHandlingWrapperDoFn</span><span class="o">.</span><span class="vm">__dict__</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">object</span><span class="o">.</span><span class="fm">__getattribute__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">,</span> <span class="n">name</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="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">try</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">_fn</span><span class="o">.</span><span class="n">process</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">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_partial</span><span class="p">:</span>
<span class="c1"># Don&#39;t emit any results until we know there will be no errors.</span>
<span class="n">result</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">result</span><span class="p">)</span>
<span class="k">yield from</span> <span class="n">result</span>
<span class="k">except</span> <span class="bp">self</span><span class="o">.</span><span class="n">_exc_class</span> <span class="k">as</span> <span class="n">exn</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">TaggedOutput</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_dead_letter_tag</span><span class="p">,</span>
<span class="p">(</span>
<span class="n">args</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="p">(</span>
<span class="nb">type</span><span class="p">(</span><span class="n">exn</span><span class="p">),</span>
<span class="nb">repr</span><span class="p">(</span><span class="n">exn</span><span class="p">),</span>
<span class="n">traceback</span><span class="o">.</span><span class="n">format_exception</span><span class="p">(</span><span class="o">*</span><span class="n">sys</span><span class="o">.</span><span class="n">exc_info</span><span class="p">()))))</span>
<span class="k">class</span> <span class="nc">_SubprocessDoFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Process method run in a subprocess, turning hard crashes into exceptions.</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">fn</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">_serialized_fn</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="n">fn</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__getattribute__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">):</span>
<span class="k">if</span> <span class="p">(</span><span class="n">name</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">&#39;__&#39;</span><span class="p">)</span> <span class="ow">or</span> <span class="n">name</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="vm">__dict__</span> <span class="ow">or</span>
<span class="n">name</span> <span class="ow">in</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="vm">__dict__</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">object</span><span class="o">.</span><span class="fm">__getattribute__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fn</span><span class="p">,</span> <span class="n">name</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="bp">self</span><span class="o">.</span><span class="n">_pool</span> <span class="o">=</span> <span class="kc">None</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="c1"># The pool is initialized lazily, including calls to setup and start_bundle.</span>
<span class="c1"># This allows us to continue processing elements after a crash.</span>
<span class="k">pass</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="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="bp">self</span><span class="o">.</span><span class="n">_call_remote</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_remote_process</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">finish_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">_call_remote</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_remote_finish_bundle</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">_call_remote</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_remote_teardown</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pool</span><span class="o">.</span><span class="n">shutdown</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pool</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">_call_remote</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">method</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">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pool</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pool</span> <span class="o">=</span> <span class="n">concurrent</span><span class="o">.</span><span class="n">futures</span><span class="o">.</span><span class="n">ProcessPoolExecutor</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pool</span><span class="o">.</span><span class="n">submit</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_remote_init</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_serialized_fn</span><span class="p">)</span><span class="o">.</span><span class="n">result</span><span class="p">()</span>
<span class="k">try</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pool</span><span class="o">.</span><span class="n">submit</span><span class="p">(</span><span class="n">method</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="o">.</span><span class="n">result</span><span class="p">()</span>
<span class="k">except</span> <span class="n">concurrent</span><span class="o">.</span><span class="n">futures</span><span class="o">.</span><span class="n">process</span><span class="o">.</span><span class="n">BrokenProcessPool</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pool</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">raise</span>
<span class="c1"># These are classmethods to avoid picking the state of self.</span>
<span class="c1"># They should only be called in an isolated process, so there&#39;s no concern</span>
<span class="c1"># about sharing state or thread safety.</span>
<span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">_remote_init</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">serialized_fn</span><span class="p">):</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_serialized_fn</span> <span class="o">=</span> <span class="n">serialized_fn</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_fn</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_started</span> <span class="o">=</span> <span class="kc">False</span>
<span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">_remote_process</span><span class="p">(</span><span class="bp">cls</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">if</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_fn</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_fn</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="bp">cls</span><span class="o">.</span><span class="n">_serialized_fn</span><span class="p">)</span>
<span class="bp">cls</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="k">if</span> <span class="ow">not</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_started</span><span class="p">:</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_fn</span><span class="o">.</span><span class="n">start_bundle</span><span class="p">()</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_started</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">result</span> <span class="o">=</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_fn</span><span class="o">.</span><span class="n">process</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">if</span> <span class="n">result</span><span class="p">:</span>
<span class="c1"># Don&#39;t return generator objects.</span>
<span class="n">result</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">result</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span>
<span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">_remote_finish_bundle</span><span class="p">(</span><span class="bp">cls</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_started</span><span class="p">:</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_started</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">if</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_fn</span><span class="o">.</span><span class="n">finish_bundle</span><span class="p">():</span>
<span class="c1"># This is because we restart and re-initialize the pool if it crashed.</span>
<span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span>
<span class="s2">&quot;Returning elements from _SubprocessDoFn.finish_bundle not safe.&quot;</span><span class="p">)</span>
<span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">_remote_teardown</span><span class="p">(</span><span class="bp">cls</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_fn</span><span class="p">:</span>
<span class="bp">cls</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="bp">cls</span><span class="o">.</span><span class="n">_fn</span> <span class="o">=</span> <span class="kc">None</span>
<div class="viewcode-block" id="Filter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Filter">[docs]</a><span class="k">def</span> <span class="nf">Filter</span><span class="p">(</span><span class="n">fn</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"># pylint: disable=invalid-name</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;:func:`Filter` is a :func:`FlatMap` with its callable filtering out</span>
<span class="sd"> elements.</span>
<span class="sd"> Filter accepts a function that keeps elements that return True, and filters</span>
<span class="sd"> out the remaining elements.</span>
<span class="sd"> Args:</span>
<span class="sd"> fn (``Callable[..., bool]``): a callable object. First argument will be an</span>
<span class="sd"> element.</span>
<span class="sd"> *args: positional arguments passed to the transform callable.</span>
<span class="sd"> **kwargs: keyword arguments passed to the transform callable.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ~apache_beam.pvalue.PCollection:</span>
<span class="sd"> A :class:`~apache_beam.pvalue.PCollection` containing the</span>
<span class="sd"> :func:`Filter` outputs.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: If the **fn** passed as argument is not a callable.</span>
<span class="sd"> Typical error is to pass a :class:`DoFn` instance which is supported only</span>
<span class="sd"> for :class:`ParDo`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;Filter can be used only with callable objects. &#39;</span>
<span class="s1">&#39;Received </span><span class="si">%r</span><span class="s1"> instead.&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">fn</span><span class="p">))</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</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="p">[</span><span class="n">x</span><span class="p">]</span> <span class="k">if</span> <span class="n">fn</span><span class="p">(</span><span class="n">x</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">else</span> <span class="p">[]</span>
<span class="n">label</span> <span class="o">=</span> <span class="s1">&#39;Filter(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="c1"># TODO: What about callable classes?</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="s1">&#39;__name__&#39;</span><span class="p">):</span>
<span class="n">wrapper</span><span class="o">.</span><span class="vm">__name__</span> <span class="o">=</span> <span class="n">fn</span><span class="o">.</span><span class="vm">__name__</span>
<span class="c1"># Get type hints from this instance or the callable. Do not use output type</span>
<span class="c1"># hints from the callable (which should be bool if set).</span>
<span class="n">fn_type_hints</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">decorators</span><span class="o">.</span><span class="n">IOTypeHints</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="k">if</span> <span class="n">fn_type_hints</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">fn_type_hints</span> <span class="o">=</span> <span class="n">fn_type_hints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">()</span>
<span class="n">type_hints</span> <span class="o">=</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span><span class="o">.</span><span class="n">with_defaults</span><span class="p">(</span><span class="n">fn_type_hints</span><span class="p">)</span>
<span class="c1"># Proxy the type-hint information from the function being wrapped, setting the</span>
<span class="c1"># output type to be the same as the input type.</span>
<span class="k">if</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="n">with_input_types</span><span class="p">(</span>
<span class="o">*</span><span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="o">**</span><span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span><span class="p">[</span><span class="mi">1</span><span class="p">])(</span>
<span class="n">wrapper</span><span class="p">)</span>
<span class="n">output_hint</span> <span class="o">=</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">simple_output_type</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="n">output_hint</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="n">wrapper</span><span class="p">)</span><span class="o">.</span><span class="n">input_types</span> <span class="ow">and</span>
<span class="n">get_type_hints</span><span class="p">(</span><span class="n">wrapper</span><span class="p">)</span><span class="o">.</span><span class="n">input_types</span><span class="p">[</span><span class="mi">0</span><span class="p">]):</span>
<span class="n">output_hint</span> <span class="o">=</span> <span class="n">get_type_hints</span><span class="p">(</span><span class="n">wrapper</span><span class="p">)</span><span class="o">.</span><span class="n">input_types</span><span class="p">[</span><span class="mi">0</span><span class="p">][</span><span class="mi">0</span><span class="p">]</span>
<span class="k">if</span> <span class="n">output_hint</span><span class="p">:</span>
<span class="n">wrapper</span> <span class="o">=</span> <span class="n">with_output_types</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">_strip_output_annotations</span><span class="p">(</span><span class="n">output_hint</span><span class="p">)])(</span>
<span class="n">wrapper</span><span class="p">)</span>
<span class="c1"># pylint: disable=protected-access</span>
<span class="n">wrapper</span><span class="o">.</span><span class="n">_argspec_fn</span> <span class="o">=</span> <span class="n">fn</span>
<span class="c1"># pylint: enable=protected-access</span>
<span class="n">pardo</span> <span class="o">=</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">wrapper</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">pardo</span><span class="o">.</span><span class="n">label</span> <span class="o">=</span> <span class="n">label</span>
<span class="k">return</span> <span class="n">pardo</span></div>
<span class="k">def</span> <span class="nf">_combine_payload</span><span class="p">(</span><span class="n">combine_fn</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">CombinePayload</span><span class="p">(</span>
<span class="n">combine_fn</span><span class="o">=</span><span class="n">combine_fn</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="n">accumulator_coder_id</span><span class="o">=</span><span class="n">context</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span>
<span class="n">combine_fn</span><span class="o">.</span><span class="n">get_accumulator_coder</span><span class="p">()))</span>
<div class="viewcode-block" id="CombineGlobally"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally">[docs]</a><span class="k">class</span> <span class="nc">CombineGlobally</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A :class:`CombineGlobally` transform.</span>
<span class="sd"> Reduces a :class:`~apache_beam.pvalue.PCollection` to a single value by</span>
<span class="sd"> progressively applying a :class:`CombineFn` to portions of the</span>
<span class="sd"> :class:`~apache_beam.pvalue.PCollection` (and to intermediate values created</span>
<span class="sd"> thereby). See documentation in :class:`CombineFn` for details on the specifics</span>
<span class="sd"> on how :class:`CombineFn` s are applied.</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll (~apache_beam.pvalue.PCollection):</span>
<span class="sd"> a :class:`~apache_beam.pvalue.PCollection` to be reduced into a single</span>
<span class="sd"> value.</span>
<span class="sd"> fn (callable): a :class:`CombineFn` object that will be called to</span>
<span class="sd"> progressively reduce the :class:`~apache_beam.pvalue.PCollection` into</span>
<span class="sd"> single values, or a callable suitable for wrapping by</span>
<span class="sd"> :class:`~apache_beam.transforms.core.CallableWrapperCombineFn`.</span>
<span class="sd"> *args: positional arguments passed to the :class:`CombineFn` object.</span>
<span class="sd"> **kwargs: keyword arguments passed to the :class:`CombineFn` object.</span>
<span class="sd"> Raises:</span>
<span class="sd"> TypeError: If the output type of the input</span>
<span class="sd"> :class:`~apache_beam.pvalue.PCollection` is not compatible</span>
<span class="sd"> with ``Iterable[A]``.</span>
<span class="sd"> Returns:</span>
<span class="sd"> ~apache_beam.pvalue.PCollection: A single-element</span>
<span class="sd"> :class:`~apache_beam.pvalue.PCollection` containing the main output of</span>
<span class="sd"> the :class:`CombineGlobally` transform.</span>
<span class="sd"> Note that the positional and keyword arguments will be processed in order</span>
<span class="sd"> to detect :class:`~apache_beam.pvalue.PValue` s that will be computed as side</span>
<span class="sd"> inputs to the transform.</span>
<span class="sd"> During pipeline execution whenever the :class:`CombineFn` object gets executed</span>
<span class="sd"> (i.e. any of the :class:`CombineFn` methods get called), the</span>
<span class="sd"> :class:`~apache_beam.pvalue.PValue` arguments will be replaced by their</span>
<span class="sd"> actual value in the exact position where they appear in the argument lists.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">has_defaults</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">as_view</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">fanout</span> <span class="o">=</span> <span class="kc">None</span> <span class="c1"># type: typing.Optional[int]</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="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">if</span> <span class="ow">not</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">CombineFn</span><span class="p">)</span> <span class="ow">or</span> <span class="nb">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;CombineGlobally can be used only with combineFn objects. &#39;</span>
<span class="s1">&#39;Received </span><span class="si">%r</span><span class="s1"> instead.&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">fn</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">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>
<div class="viewcode-block" id="CombineGlobally.display_data"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally.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;combine_fn&#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">fn</span><span class="o">.</span><span class="vm">__class__</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Combine Function&#39;</span><span class="p">),</span>
<span class="s1">&#39;combine_fn_dd&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span>
<span class="p">}</span></div>
<div class="viewcode-block" id="CombineGlobally.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally.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">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">fanout</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="s1">&#39;</span><span class="si">%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="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="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="s1">&#39;</span><span class="si">%s</span><span class="s1">(</span><span class="si">%s</span><span class="s1">, fanout=</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="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">fanout</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_clone</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">extra_attributes</span><span class="p">):</span>
<span class="n">clone</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">clone</span><span class="o">.</span><span class="vm">__dict__</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">extra_attributes</span><span class="p">)</span>
<span class="k">return</span> <span class="n">clone</span>
<div class="viewcode-block" id="CombineGlobally.with_fanout"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally.with_fanout">[docs]</a> <span class="k">def</span> <span class="nf">with_fanout</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">fanout</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_clone</span><span class="p">(</span><span class="n">fanout</span><span class="o">=</span><span class="n">fanout</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombineGlobally.with_defaults"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally.with_defaults">[docs]</a> <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="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_clone</span><span class="p">(</span><span class="n">has_defaults</span><span class="o">=</span><span class="n">has_defaults</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombineGlobally.without_defaults"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally.without_defaults">[docs]</a> <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>
<div class="viewcode-block" id="CombineGlobally.as_singleton_view"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally.as_singleton_view">[docs]</a> <span class="k">def</span> <span class="nf">as_singleton_view</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">_clone</span><span class="p">(</span><span class="n">as_view</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombineGlobally.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally.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">def</span> <span class="nf">add_input_types</span><span class="p">(</span><span class="n">transform</span><span class="p">):</span>
<span class="n">type_hints</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_type_hints</span><span class="p">()</span>
<span class="k">if</span> <span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span><span class="p">:</span>
<span class="k">return</span> <span class="n">transform</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">type_hints</span><span class="o">.</span><span class="n">input_types</span><span class="p">[</span><span class="mi">0</span><span class="p">][</span><span class="mi">0</span><span class="p">])</span>
<span class="k">return</span> <span class="n">transform</span>
<span class="n">combine_fn</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="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="n">has_side_inputs</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="n">combine_per_key</span> <span class="o">=</span> <span class="n">CombinePerKey</span><span class="p">(</span><span class="n">combine_fn</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">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">fanout</span><span class="p">:</span>
<span class="n">combine_per_key</span> <span class="o">=</span> <span class="n">combine_per_key</span><span class="o">.</span><span class="n">with_hot_key_fanout</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fanout</span><span class="p">)</span>
<span class="n">combined</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">pcoll</span>
<span class="o">|</span> <span class="s1">&#39;KeyWithVoid&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">add_input_types</span><span class="p">(</span>
<span class="n">ParDo</span><span class="p">(</span><span class="n">_KeyWithNone</span><span class="p">())</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span><span class="kc">None</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="o">|</span> <span class="s1">&#39;CombinePerKey&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">combine_per_key</span>
<span class="o">|</span> <span class="s1">&#39;UnKey&#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">k_v</span><span class="p">:</span> <span class="n">k_v</span><span class="p">[</span><span class="mi">1</span><span class="p">]))</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">has_defaults</span> <span class="ow">and</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">as_view</span><span class="p">:</span>
<span class="k">return</span> <span class="n">combined</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">as_view</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">try</span><span class="p">:</span>
<span class="n">combine_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="c1"># This is called in the main program, but cannot be avoided</span>
<span class="c1"># in the as_view case as it must be available to all windows.</span>
<span class="n">default_value</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="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">finally</span><span class="p">:</span>
<span class="n">combine_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">else</span><span class="p">:</span>
<span class="n">default_value</span> <span class="o">=</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">AsSingleton</span><span class="o">.</span><span class="n">_NO_DEFAULT</span>
<span class="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">AsSingleton</span><span class="p">(</span><span class="n">combined</span><span class="p">,</span> <span class="n">default_value</span><span class="o">=</span><span class="n">default_value</span><span class="p">)</span>
<span class="k">else</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">windowfn</span> <span class="o">!=</span> <span class="n">GlobalWindows</span><span class="p">():</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Default values are not yet supported in CombineGlobally() if the &quot;</span>
<span class="s2">&quot;output PCollection is not windowed by GlobalWindows. &quot;</span>
<span class="s2">&quot;Instead, use CombineGlobally().without_defaults() to output &quot;</span>
<span class="s2">&quot;an empty PCollection if the input PCollection is empty, &quot;</span>
<span class="s2">&quot;or CombineGlobally().as_singleton_view() to get the default &quot;</span>
<span class="s2">&quot;output of the CombineFn if the input PCollection is empty.&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">typed</span><span class="p">(</span><span class="n">transform</span><span class="p">):</span>
<span class="c1"># TODO(robertwb): We should infer this.</span>
<span class="k">if</span> <span class="n">combined</span><span class="o">.</span><span class="n">element_type</span><span class="p">:</span>
<span class="k">return</span> <span class="n">transform</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">combined</span><span class="o">.</span><span class="n">element_type</span><span class="p">)</span>
<span class="k">return</span> <span class="n">transform</span>
<span class="c1"># Capture in closure (avoiding capturing self).</span>
<span class="n">args</span><span class="p">,</span> <span class="n">kwargs</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="bp">self</span><span class="o">.</span><span class="n">kwargs</span>
<span class="k">def</span> <span class="nf">inject_default</span><span class="p">(</span><span class="n">_</span><span class="p">,</span> <span class="n">combined</span><span class="p">):</span>
<span class="k">if</span> <span class="n">combined</span><span class="p">:</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">combined</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span>
<span class="k">return</span> <span class="n">combined</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">combine_fn</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="n">default</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="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">finally</span><span class="p">:</span>
<span class="n">combine_fn</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">return</span> <span class="n">default</span>
<span class="k">return</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="s1">&#39;DoOnce&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">Create</span><span class="p">([</span><span class="kc">None</span><span class="p">])</span>
<span class="o">|</span> <span class="s1">&#39;InjectDefault&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">typed</span><span class="p">(</span>
<span class="n">Map</span><span class="p">(</span><span class="n">inject_default</span><span class="p">,</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">AsList</span><span class="p">(</span><span class="n">combined</span><span class="p">))))</span></div>
<div class="viewcode-block" id="CombineGlobally.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineGlobally.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">composites</span><span class="o">.</span><span class="n">COMBINE_GLOBALLY</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">CombinePayload</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">combine_payload</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">CombineGlobally</span><span class="p">(</span>
<span class="n">CombineFn</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">combine_payload</span><span class="o">.</span><span class="n">combine_fn</span><span class="p">,</span> <span class="n">context</span><span class="p">))</span></div></div>
<span class="nd">@DoFnInfo</span><span class="o">.</span><span class="n">register_stateless_dofn</span><span class="p">(</span><span class="n">python_urns</span><span class="o">.</span><span class="n">KEY_WITH_NONE_DOFN</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_KeyWithNone</span><span class="p">(</span><span class="n">DoFn</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">v</span><span class="p">):</span>
<span class="k">yield</span> <span class="kc">None</span><span class="p">,</span> <span class="n">v</span>
<div class="viewcode-block" id="CombinePerKey"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey">[docs]</a><span class="k">class</span> <span class="nc">CombinePerKey</span><span class="p">(</span><span class="n">PTransformWithSideInputs</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A per-key Combine transform.</span>
<span class="sd"> Identifies sets of values associated with the same key in the input</span>
<span class="sd"> PCollection, then applies a CombineFn to condense those sets to single</span>
<span class="sd"> values. See documentation in CombineFn for details on the specifics on how</span>
<span class="sd"> CombineFns are applied.</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll: input pcollection.</span>
<span class="sd"> fn: instance of CombineFn to apply to all values under the same key in</span>
<span class="sd"> pcoll, or a callable whose signature is ``f(iterable, *args, **kwargs)``</span>
<span class="sd"> (e.g., sum, max).</span>
<span class="sd"> *args: arguments and side inputs, passed directly to the CombineFn.</span>
<span class="sd"> **kwargs: arguments and side inputs, passed directly to the CombineFn.</span>
<span class="sd"> Returns:</span>
<span class="sd"> A PObject holding the result of the combine operation.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="CombinePerKey.with_hot_key_fanout"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.with_hot_key_fanout">[docs]</a> <span class="k">def</span> <span class="nf">with_hot_key_fanout</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">fanout</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A per-key combine operation like self but with two levels of aggregation.</span>
<span class="sd"> If a given key is produced by too many upstream bundles, the final</span>
<span class="sd"> reduction can become a bottleneck despite partial combining being lifted</span>
<span class="sd"> pre-GroupByKey. In these cases it can be helpful to perform intermediate</span>
<span class="sd"> partial aggregations in parallel and then re-group to peform a final</span>
<span class="sd"> (per-key) combine. This is also useful for high-volume keys in streaming</span>
<span class="sd"> where combiners are not generally lifted for latency reasons.</span>
<span class="sd"> Note that a fanout greater than 1 requires the data to be sent through</span>
<span class="sd"> two GroupByKeys, and a high fanout can also result in more shuffle data</span>
<span class="sd"> due to less per-bundle combining. Setting the fanout for a key at 1 or less</span>
<span class="sd"> places values on the &quot;cold key&quot; path that skip the intermediate level of</span>
<span class="sd"> aggregation.</span>
<span class="sd"> Args:</span>
<span class="sd"> fanout: either None, for no fanout, an int, for a constant-degree fanout,</span>
<span class="sd"> or a callable mapping keys to a key-specific degree of fanout.</span>
<span class="sd"> Returns:</span>
<span class="sd"> A per-key combining PTransform with the specified fanout.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.combiners</span> <span class="kn">import</span> <span class="n">curry_combine_fn</span>
<span class="k">if</span> <span class="n">fanout</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">_CombinePerKeyWithHotKeyFanout</span><span class="p">(</span>
<span class="n">curry_combine_fn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">),</span> <span class="n">fanout</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombinePerKey.display_data"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.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;combine_fn&#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">fn</span><span class="o">.</span><span class="vm">__class__</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">&#39;Combine Function&#39;</span><span class="p">),</span>
<span class="s1">&#39;combine_fn_dd&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span>
<span class="p">}</span></div>
<div class="viewcode-block" id="CombinePerKey.make_fn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.make_fn">[docs]</a> <span class="k">def</span> <span class="nf">make_fn</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">has_side_inputs</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_fn_label</span> <span class="o">=</span> <span class="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span>
<span class="k">return</span> <span class="n">CombineFn</span><span class="o">.</span><span class="n">maybe_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">has_side_inputs</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombinePerKey.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.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;</span><span class="si">%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="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">_fn_label</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_process_argspec_fn</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="k">lambda</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="kc">None</span>
<div class="viewcode-block" id="CombinePerKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.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">args</span><span class="p">,</span> <span class="n">kwargs</span> <span class="o">=</span> <span class="n">util</span><span class="o">.</span><span class="n">insert_values_in_args</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">GroupByKey</span><span class="p">()</span> <span class="o">|</span> <span class="s1">&#39;Combine&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">CombineValues</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">fn</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></div>
<div class="viewcode-block" id="CombinePerKey.default_type_hints"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.default_type_hints">[docs]</a> <span class="k">def</span> <span class="nf">default_type_hints</span><span class="p">(</span><span class="bp">self</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">fn</span><span class="o">.</span><span class="n">get_type_hints</span><span class="p">()</span>
<span class="n">k</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">TypeVariable</span><span class="p">(</span><span class="s1">&#39;K&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">result</span><span class="o">.</span><span class="n">input_types</span><span class="p">:</span>
<span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span> <span class="o">=</span> <span class="n">result</span><span class="o">.</span><span class="n">input_types</span>
<span class="n">args</span> <span class="o">=</span> <span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">k</span><span class="p">,</span> <span class="n">args</span><span class="p">[</span><span class="mi">0</span><span class="p">]],</span> <span class="p">)</span> <span class="o">+</span> <span class="n">args</span><span class="p">[</span><span class="mi">1</span><span class="p">:]</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">result</span><span class="o">.</span><span class="n">with_input_types</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">else</span><span class="p">:</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">result</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">k</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">])</span>
<span class="k">if</span> <span class="n">result</span><span class="o">.</span><span class="n">output_types</span><span class="p">:</span>
<span class="n">main_output_type</span> <span class="o">=</span> <span class="n">result</span><span class="o">.</span><span class="n">simple_output_type</span><span class="p">(</span><span class="s1">&#39;&#39;</span><span class="p">)</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">result</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">k</span><span class="p">,</span> <span class="n">main_output_type</span><span class="p">])</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">result</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">k</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">])</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="CombinePerKey.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</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="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; typing.Tuple[str, beam_runner_api_pb2.CombinePayload]</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">args</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.combiners</span> <span class="kn">import</span> <span class="n">curry_combine_fn</span>
<span class="n">combine_fn</span> <span class="o">=</span> <span class="n">curry_combine_fn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">combine_fn</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">composites</span><span class="o">.</span><span class="n">COMBINE_PER_KEY</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">_combine_payload</span><span class="p">(</span><span class="n">combine_fn</span><span class="p">,</span> <span class="n">context</span><span class="p">))</span></div>
<div class="viewcode-block" id="CombinePerKey.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">composites</span><span class="o">.</span><span class="n">COMBINE_PER_KEY</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">CombinePayload</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">combine_payload</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">CombinePerKey</span><span class="p">(</span>
<span class="n">CombineFn</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">combine_payload</span><span class="o">.</span><span class="n">combine_fn</span><span class="p">,</span> <span class="n">context</span><span class="p">))</span></div>
<div class="viewcode-block" id="CombinePerKey.runner_api_requires_keyed_input"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombinePerKey.runner_api_requires_keyed_input">[docs]</a> <span class="k">def</span> <span class="nf">runner_api_requires_keyed_input</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span></div></div>
<span class="c1"># TODO(robertwb): Rename to CombineGroupedValues?</span>
<div class="viewcode-block" id="CombineValues"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineValues">[docs]</a><span class="k">class</span> <span class="nc">CombineValues</span><span class="p">(</span><span class="n">PTransformWithSideInputs</span><span class="p">):</span>
<div class="viewcode-block" id="CombineValues.make_fn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineValues.make_fn">[docs]</a> <span class="k">def</span> <span class="nf">make_fn</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">has_side_inputs</span><span class="p">):</span>
<span class="k">return</span> <span class="n">CombineFn</span><span class="o">.</span><span class="n">maybe_from_callable</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">has_side_inputs</span><span class="p">)</span></div>
<div class="viewcode-block" id="CombineValues.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineValues.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">args</span><span class="p">,</span> <span class="n">kwargs</span> <span class="o">=</span> <span class="n">util</span><span class="o">.</span><span class="n">insert_values_in_args</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">)</span>
<span class="n">input_type</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span>
<span class="n">key_type</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">input_type</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">key_type</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">input_type</span><span class="o">.</span><span class="n">tuple_types</span>
<span class="n">runtime_type_check</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">_options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">TypeOptions</span><span class="p">)</span><span class="o">.</span><span class="n">runtime_type_check</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span>
<span class="n">CombineValuesDoFn</span><span class="p">(</span><span class="n">key_type</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="n">runtime_type_check</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></div>
<div class="viewcode-block" id="CombineValues.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineValues.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</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">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">args</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.combiners</span> <span class="kn">import</span> <span class="n">curry_combine_fn</span>
<span class="n">combine_fn</span> <span class="o">=</span> <span class="n">curry_combine_fn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">combine_fn</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">combine_components</span><span class="o">.</span><span class="n">COMBINE_GROUPED_VALUES</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">_combine_payload</span><span class="p">(</span><span class="n">combine_fn</span><span class="p">,</span> <span class="n">context</span><span class="p">))</span></div>
<div class="viewcode-block" id="CombineValues.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.CombineValues.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">combine_components</span><span class="o">.</span><span class="n">COMBINE_GROUPED_VALUES</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">CombinePayload</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">combine_payload</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">CombineValues</span><span class="p">(</span>
<span class="n">CombineFn</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">combine_payload</span><span class="o">.</span><span class="n">combine_fn</span><span class="p">,</span> <span class="n">context</span><span class="p">))</span></div></div>
<span class="k">class</span> <span class="nc">CombineValuesDoFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;DoFn for performing per-key Combine 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">input_pcoll_type</span><span class="p">,</span>
<span class="n">combinefn</span><span class="p">,</span> <span class="c1"># type: CombineFn</span>
<span class="n">runtime_type_check</span><span class="p">,</span> <span class="c1"># type: bool</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">combinefn</span> <span class="o">=</span> <span class="n">combinefn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">runtime_type_check</span> <span class="o">=</span> <span class="n">runtime_type_check</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">combinefn</span><span class="o">.</span><span class="n">setup</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="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"># Expected elements input to this DoFn are 2-tuples of the form</span>
<span class="c1"># (key, iter), with iter an iterable of all the values associated with key</span>
<span class="c1"># in the input PCollection.</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">runtime_type_check</span><span class="p">:</span>
<span class="c1"># Apply the combiner in a single operation rather than artificially</span>
<span class="c1"># breaking it up so that output type violations manifest as TypeCheck</span>
<span class="c1"># errors rather than type errors.</span>
<span class="k">return</span> <span class="p">[(</span><span class="n">element</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="bp">self</span><span class="o">.</span><span class="n">combinefn</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">element</span><span class="p">[</span><span class="mi">1</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"># Add the elements into three accumulators (for testing of merge).</span>
<span class="n">elements</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">element</span><span class="p">[</span><span class="mi">1</span><span class="p">])</span>
<span class="n">accumulators</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">k</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">3</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">elements</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="n">k</span><span class="p">:</span>
<span class="k">break</span>
<span class="n">accumulators</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">combinefn</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">combinefn</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="n">elements</span><span class="p">[</span><span class="n">k</span><span class="p">::</span><span class="mi">3</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"># Merge the accumulators.</span>
<span class="n">accumulator</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">combinefn</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="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="c1"># Convert accumulator to the final result.</span>
<span class="k">return</span> <span class="p">[(</span>
<span class="n">element</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="bp">self</span><span class="o">.</span><span class="n">combinefn</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="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">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">combinefn</span><span class="o">.</span><span class="n">teardown</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">default_type_hints</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">hints</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">combinefn</span><span class="o">.</span><span class="n">get_type_hints</span><span class="p">()</span>
<span class="k">if</span> <span class="n">hints</span><span class="o">.</span><span class="n">input_types</span><span class="p">:</span>
<span class="n">K</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">TypeVariable</span><span class="p">(</span><span class="s1">&#39;K&#39;</span><span class="p">)</span>
<span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span> <span class="o">=</span> <span class="n">hints</span><span class="o">.</span><span class="n">input_types</span>
<span class="n">args</span> <span class="o">=</span> <span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">args</span><span class="p">[</span><span class="mi">0</span><span class="p">]]],</span> <span class="p">)</span> <span class="o">+</span> <span class="n">args</span><span class="p">[</span><span class="mi">1</span><span class="p">:]</span>
<span class="n">hints</span> <span class="o">=</span> <span class="n">hints</span><span class="o">.</span><span class="n">with_input_types</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">else</span><span class="p">:</span>
<span class="n">K</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span>
<span class="k">if</span> <span class="n">hints</span><span class="o">.</span><span class="n">output_types</span><span class="p">:</span>
<span class="n">main_output_type</span> <span class="o">=</span> <span class="n">hints</span><span class="o">.</span><span class="n">simple_output_type</span><span class="p">(</span><span class="s1">&#39;&#39;</span><span class="p">)</span>
<span class="n">hints</span> <span class="o">=</span> <span class="n">hints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">main_output_type</span><span class="p">])</span>
<span class="k">return</span> <span class="n">hints</span>
<span class="k">class</span> <span class="nc">_CombinePerKeyWithHotKeyFanout</span><span class="p">(</span><span class="n">PTransform</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">combine_fn</span><span class="p">,</span> <span class="c1"># type: CombineFn</span>
<span class="n">fanout</span><span class="p">,</span> <span class="c1"># type: typing.Union[int, typing.Callable[[typing.Any], int]]</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_combine_fn</span> <span class="o">=</span> <span class="n">combine_fn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_fanout_fn</span> <span class="o">=</span> <span class="p">((</span><span class="k">lambda</span> <span class="n">key</span><span class="p">:</span> <span class="n">fanout</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">fanout</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="k">else</span> <span class="n">fanout</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;</span><span class="si">%s</span><span class="s1">(</span><span class="si">%s</span><span class="s1">, fanout=</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="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_combine_fn</span><span class="p">),</span>
<span class="n">ptransform</span><span class="o">.</span><span class="n">label_from_callable</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fanout_fn</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">AccumulationMode</span>
<span class="n">combine_fn</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_combine_fn</span>
<span class="n">fanout_fn</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fanout_fn</span>
<span class="k">if</span> <span class="nb">isinstance</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="n">SlidingWindows</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;CombinePerKey.with_hot_key_fanout does not yet work properly with &#39;</span>
<span class="s1">&#39;SlidingWindows. See: https://github.com/apache/beam/issues/20528&#39;</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">SplitHotCold</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</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="c1"># Spreading a hot key across all possible sub-keys for all bundles</span>
<span class="c1"># would defeat the goal of not overwhelming downstream reducers</span>
<span class="c1"># (as well as making less efficient use of PGBK combining tables).</span>
<span class="c1"># Instead, each bundle independently makes a consistent choice about</span>
<span class="c1"># which &quot;shard&quot; of a key to send its intermediate results.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_nonce</span> <span class="o">=</span> <span class="nb">int</span><span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">getrandbits</span><span class="p">(</span><span class="mi">31</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">):</span>
<span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">element</span>
<span class="n">fanout</span> <span class="o">=</span> <span class="n">fanout_fn</span><span class="p">(</span><span class="n">key</span><span class="p">)</span>
<span class="k">if</span> <span class="n">fanout</span> <span class="o">&lt;=</span> <span class="mi">1</span><span class="p">:</span>
<span class="c1"># Boolean indicates this is not an accumulator.</span>
<span class="k">yield</span> <span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="p">(</span><span class="kc">False</span><span class="p">,</span> <span class="n">value</span><span class="p">))</span> <span class="c1"># cold</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">TaggedOutput</span><span class="p">(</span><span class="s1">&#39;hot&#39;</span><span class="p">,</span> <span class="p">((</span><span class="bp">self</span><span class="o">.</span><span class="n">_nonce</span> <span class="o">%</span> <span class="n">fanout</span><span class="p">,</span> <span class="n">key</span><span class="p">),</span> <span class="n">value</span><span class="p">))</span>
<span class="k">class</span> <span class="nc">PreCombineFn</span><span class="p">(</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">extract_output</span><span class="p">(</span><span class="n">accumulator</span><span class="p">):</span>
<span class="c1"># Boolean indicates this is an accumulator.</span>
<span class="k">return</span> <span class="p">(</span><span class="kc">True</span><span class="p">,</span> <span class="n">accumulator</span><span class="p">)</span>
<span class="n">setup</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">setup</span>
<span class="n">create_accumulator</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">create_accumulator</span>
<span class="n">add_input</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">add_input</span>
<span class="n">merge_accumulators</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">merge_accumulators</span>
<span class="n">compact</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">compact</span>
<span class="n">teardown</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">teardown</span>
<span class="k">class</span> <span class="nc">PostCombineFn</span><span class="p">(</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">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="n">is_accumulator</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">element</span>
<span class="k">if</span> <span class="n">is_accumulator</span><span class="p">:</span>
<span class="k">return</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">merge_accumulators</span><span class="p">([</span><span class="n">accumulator</span><span class="p">,</span> <span class="n">value</span><span class="p">])</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">combine_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">value</span><span class="p">)</span>
<span class="n">setup</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">setup</span>
<span class="n">create_accumulator</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">create_accumulator</span>
<span class="n">merge_accumulators</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">merge_accumulators</span>
<span class="n">compact</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">compact</span>
<span class="n">extract_output</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">extract_output</span>
<span class="n">teardown</span> <span class="o">=</span> <span class="n">combine_fn</span><span class="o">.</span><span class="n">teardown</span>
<span class="k">def</span> <span class="nf">StripNonce</span><span class="p">(</span><span class="n">nonce_key_value</span><span class="p">):</span>
<span class="p">(</span><span class="n">_</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">nonce_key_value</span>
<span class="k">return</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span>
<span class="n">cold</span><span class="p">,</span> <span class="n">hot</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">SplitHotCold</span><span class="p">())</span><span class="o">.</span><span class="n">with_outputs</span><span class="p">(</span><span class="s1">&#39;hot&#39;</span><span class="p">,</span> <span class="n">main</span><span class="o">=</span><span class="s1">&#39;cold&#39;</span><span class="p">)</span>
<span class="n">cold</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span> <span class="c1"># No multi-output type hints.</span>
<span class="n">precombined_hot</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">hot</span>
<span class="c1"># Avoid double counting that may happen with stacked accumulating mode.</span>
<span class="o">|</span> <span class="s1">&#39;WindowIntoDiscarding&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">WindowInto</span><span class="p">(</span>
<span class="n">pcoll</span><span class="o">.</span><span class="n">windowing</span><span class="p">,</span> <span class="n">accumulation_mode</span><span class="o">=</span><span class="n">AccumulationMode</span><span class="o">.</span><span class="n">DISCARDING</span><span class="p">)</span>
<span class="o">|</span> <span class="n">CombinePerKey</span><span class="p">(</span><span class="n">PreCombineFn</span><span class="p">())</span>
<span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="n">StripNonce</span><span class="p">)</span>
<span class="o">|</span> <span class="s1">&#39;WindowIntoOriginal&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">WindowInto</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">windowing</span><span class="p">))</span>
<span class="k">return</span> <span class="p">((</span><span class="n">cold</span><span class="p">,</span> <span class="n">precombined_hot</span><span class="p">)</span>
<span class="o">|</span> <span class="n">Flatten</span><span class="p">()</span>
<span class="o">|</span> <span class="n">CombinePerKey</span><span class="p">(</span><span class="n">PostCombineFn</span><span class="p">()))</span>
<div class="viewcode-block" id="GroupByKey"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">typing</span><span class="o">.</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">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">typing</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">typing</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">V</span><span class="p">]])</span>
<span class="k">class</span> <span class="nc">GroupByKey</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A group by key transform.</span>
<span class="sd"> Processes an input PCollection consisting of key/value pairs represented as a</span>
<span class="sd"> tuple pair. The result is a PCollection where values having a common key are</span>
<span class="sd"> grouped together. For example (a, 1), (b, 2), (a, 3) will result into</span>
<span class="sd"> (a, [1, 3]), (b, [2]).</span>
<span class="sd"> The implementation here is used only when run on the local direct runner.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="GroupByKey.ReifyWindows"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey.ReifyWindows">[docs]</a> <span class="k">class</span> <span class="nc">ReifyWindows</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<div class="viewcode-block" id="GroupByKey.ReifyWindows.process"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey.ReifyWindows.process">[docs]</a> <span class="k">def</span> <span class="nf">process</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">):</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="o">=</span> <span class="n">element</span>
<span class="k">except</span> <span class="ne">TypeError</span><span class="p">:</span>
<span class="k">raise</span> <span class="n">TypeCheckError</span><span class="p">(</span>
<span class="s1">&#39;Input to GroupByKey must be a PCollection with &#39;</span>
<span class="s1">&#39;elements compatible with KV[A, B]&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="p">[(</span><span class="n">k</span><span class="p">,</span> <span class="n">WindowedValue</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="p">[</span><span class="n">window</span><span class="p">]))]</span></div>
<div class="viewcode-block" id="GroupByKey.ReifyWindows.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey.ReifyWindows.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_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="n">key_type</span><span class="p">,</span> <span class="n">value_type</span> <span class="o">=</span> <span class="n">trivial_inference</span><span class="o">.</span><span class="n">key_value_types</span><span class="p">(</span><span class="n">input_type</span><span class="p">)</span>
<span class="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span>
<span class="n">key_type</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">[</span><span class="n">value_type</span><span class="p">]]</span> <span class="c1"># type: ignore[misc]</span></div></div>
<div class="viewcode-block" id="GroupByKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey.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="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">DataLossReason</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">DefaultTrigger</span>
<span class="n">windowing</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">windowing</span>
<span class="n">trigger</span> <span class="o">=</span> <span class="n">windowing</span><span class="o">.</span><span class="n">triggerfn</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">is_bounded</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">windowing</span><span class="o">.</span><span class="n">windowfn</span><span class="p">,</span> <span class="n">GlobalWindows</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">trigger</span><span class="p">,</span>
<span class="n">DefaultTrigger</span><span class="p">):</span>
<span class="k">if</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">allow_unsafe_triggers</span><span class="p">:</span>
<span class="c1"># TODO(BEAM-9487) Change comment for Beam 2.33</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;</span><span class="si">%s</span><span class="s1">: PCollection passed to GroupByKey is unbounded, has a global &#39;</span>
<span class="s1">&#39;window, and uses a default trigger. This is being allowed &#39;</span>
<span class="s1">&#39;because --allow_unsafe_triggers is set, but it may prevent &#39;</span>
<span class="s1">&#39;data from making it through the pipeline.&#39;</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">label</span><span class="p">)</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;GroupByKey cannot be applied to an unbounded &#39;</span> <span class="o">+</span>
<span class="s1">&#39;PCollection with global windowing and a default trigger&#39;</span><span class="p">)</span>
<span class="n">unsafe_reason</span> <span class="o">=</span> <span class="n">trigger</span><span class="o">.</span><span class="n">may_lose_data</span><span class="p">(</span><span class="n">windowing</span><span class="p">)</span>
<span class="k">if</span> <span class="n">unsafe_reason</span> <span class="o">!=</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span><span class="p">:</span>
<span class="n">reason_msg</span> <span class="o">=</span> <span class="nb">str</span><span class="p">(</span><span class="n">unsafe_reason</span><span class="p">)</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span><span class="s1">&#39;DataLossReason.&#39;</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">allow_unsafe_triggers</span><span class="p">:</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;</span><span class="si">%s</span><span class="s1">: Unsafe trigger `</span><span class="si">%s</span><span class="s1">` detected (reason: </span><span class="si">%s</span><span class="s1">). This is &#39;</span>
<span class="s1">&#39;being allowed because --allow_unsafe_triggers is set. This could &#39;</span>
<span class="s1">&#39;lead to missing or incomplete groups.&#39;</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">label</span><span class="p">,</span>
<span class="n">trigger</span><span class="p">,</span>
<span class="n">reason_msg</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">msg</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="si">{}</span><span class="s1">: Unsafe trigger: `</span><span class="si">{}</span><span class="s1">` may lose data. &#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">label</span><span class="p">,</span> <span class="n">trigger</span><span class="p">)</span>
<span class="n">msg</span> <span class="o">+=</span> <span class="s1">&#39;Reason: </span><span class="si">{}</span><span class="s1">. &#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">reason_msg</span><span class="p">)</span>
<span class="n">msg</span> <span class="o">+=</span> <span class="s1">&#39;This can be overriden with the --allow_unsafe_triggers flag.&#39;</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="n">msg</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="o">.</span><span class="n">from_</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span></div>
<div class="viewcode-block" id="GroupByKey.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_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="n">key_type</span><span class="p">,</span> <span class="n">value_type</span> <span class="o">=</span> <span class="p">(</span><span class="n">typehints</span><span class="o">.</span><span class="n">typehints</span><span class="o">.</span><span class="n">coerce_to_kv_type</span><span class="p">(</span>
<span class="n">input_type</span><span class="p">)</span><span class="o">.</span><span class="n">tuple_types</span><span class="p">)</span>
<span class="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span><span class="n">key_type</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">value_type</span><span class="p">]]</span></div>
<div class="viewcode-block" id="GroupByKey.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; typing.Tuple[str, None]</span>
<span class="k">return</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">GROUP_BY_KEY</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span></div>
<div class="viewcode-block" id="GroupByKey.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">GROUP_BY_KEY</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span>
<span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">unused_payload</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">GroupByKey</span><span class="p">()</span></div>
<div class="viewcode-block" id="GroupByKey.runner_api_requires_keyed_input"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupByKey.runner_api_requires_keyed_input">[docs]</a> <span class="k">def</span> <span class="nf">runner_api_requires_keyed_input</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span></div></div>
<span class="k">def</span> <span class="nf">_expr_to_callable</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">pos</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span>
<span class="k">return</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">expr</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">callable</span><span class="p">(</span><span class="n">expr</span><span class="p">):</span>
<span class="k">return</span> <span class="n">expr</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;Field expression </span><span class="si">%r</span><span class="s1"> at </span><span class="si">%s</span><span class="s1"> must be a callable or a string.&#39;</span> <span class="o">%</span>
<span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">pos</span><span class="p">))</span>
<div class="viewcode-block" id="GroupBy"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupBy">[docs]</a><span class="k">class</span> <span class="nc">GroupBy</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Groups a PCollection by one or more expressions, used to derive the key.</span>
<span class="sd"> `GroupBy(expr)` is roughly equivalent to</span>
<span class="sd"> beam.Map(lambda v: (expr(v), v)) | beam.GroupByKey()</span>
<span class="sd"> but provides several conveniences, e.g.</span>
<span class="sd"> * Several arguments may be provided, as positional or keyword arguments,</span>
<span class="sd"> resulting in a tuple-like key. For example `GroupBy(a=expr1, b=expr2)`</span>
<span class="sd"> groups by a key with attributes `a` and `b` computed by applying</span>
<span class="sd"> `expr1` and `expr2` to each element.</span>
<span class="sd"> * Strings can be used as a shorthand for accessing an attribute, e.g.</span>
<span class="sd"> `GroupBy(&#39;some_field&#39;)` is equivalent to</span>
<span class="sd"> `GroupBy(lambda v: getattr(v, &#39;some_field&#39;))`.</span>
<span class="sd"> The GroupBy operation can be made into an aggregating operation by invoking</span>
<span class="sd"> its `aggregate_field` method.</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="o">*</span><span class="n">fields</span><span class="p">,</span> <span class="c1"># type: typing.Union[str, typing.Callable]</span>
<span class="o">**</span><span class="n">kwargs</span> <span class="c1"># type: typing.Union[str, typing.Callable]</span>
<span class="p">):</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">fields</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">kwargs</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_force_tuple_keys</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">name</span> <span class="o">=</span> <span class="n">fields</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">fields</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="k">else</span> <span class="s1">&#39;key&#39;</span>
<span class="n">key_fields</span> <span class="o">=</span> <span class="p">[(</span><span class="n">name</span><span class="p">,</span> <span class="n">_expr_to_callable</span><span class="p">(</span><span class="n">fields</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="mi">0</span><span class="p">))]</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_force_tuple_keys</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">key_fields</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">ix</span><span class="p">,</span> <span class="n">field</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">fields</span><span class="p">):</span>
<span class="n">name</span> <span class="o">=</span> <span class="n">field</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">field</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="k">else</span> <span class="s1">&#39;key</span><span class="si">%d</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">ix</span>
<span class="n">key_fields</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">name</span><span class="p">,</span> <span class="n">_expr_to_callable</span><span class="p">(</span><span class="n">field</span><span class="p">,</span> <span class="n">ix</span><span class="p">)))</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">key_fields</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">name</span><span class="p">,</span> <span class="n">_expr_to_callable</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">name</span><span class="p">)))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_key_fields</span> <span class="o">=</span> <span class="n">key_fields</span>
<span class="n">field_names</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">name</span> <span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">_</span> <span class="ow">in</span> <span class="n">key_fields</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_key_type</span> <span class="o">=</span> <span class="k">lambda</span> <span class="o">*</span><span class="n">values</span><span class="p">:</span> <span class="n">_dynamic_named_tuple</span><span class="p">(</span><span class="s1">&#39;Key&#39;</span><span class="p">,</span> <span class="n">field_names</span><span class="p">)(</span>
<span class="o">*</span><span class="n">values</span><span class="p">)</span>
<div class="viewcode-block" id="GroupBy.aggregate_field"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupBy.aggregate_field">[docs]</a> <span class="k">def</span> <span class="nf">aggregate_field</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">field</span><span class="p">,</span> <span class="c1"># type: typing.Union[str, typing.Callable]</span>
<span class="n">combine_fn</span><span class="p">,</span> <span class="c1"># type: typing.Union[typing.Callable, CombineFn]</span>
<span class="n">dest</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns a grouping operation that also aggregates grouped values.</span>
<span class="sd"> Args:</span>
<span class="sd"> field: indicates the field to be aggregated</span>
<span class="sd"> combine_fn: indicates the aggregation function to be used</span>
<span class="sd"> dest: indicates the name that will be used for the aggregate in the output</span>
<span class="sd"> May be called repeatedly to aggregate multiple fields, e.g.</span>
<span class="sd"> GroupBy(&#39;key&#39;)</span>
<span class="sd"> .aggregate_field(&#39;some_attr&#39;, sum, &#39;sum_attr&#39;)</span>
<span class="sd"> .aggregate_field(lambda v: ..., MeanCombineFn, &#39;mean&#39;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">_GroupAndAggregate</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="p">())</span><span class="o">.</span><span class="n">aggregate_field</span><span class="p">(</span><span class="n">field</span><span class="p">,</span> <span class="n">combine_fn</span><span class="p">,</span> <span class="n">dest</span><span class="p">)</span></div>
<div class="viewcode-block" id="GroupBy.force_tuple_keys"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupBy.force_tuple_keys">[docs]</a> <span class="k">def</span> <span class="nf">force_tuple_keys</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Forces the keys to always be tuple-like, even if there is only a single</span>
<span class="sd"> expression.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">res</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">res</span><span class="o">.</span><span class="n">_force_tuple_keys</span> <span class="o">=</span> <span class="n">value</span>
<span class="k">return</span> <span class="n">res</span></div>
<span class="k">def</span> <span class="nf">_key_func</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="bp">self</span><span class="o">.</span><span class="n">_force_tuple_keys</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_key_fields</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="bp">self</span><span class="o">.</span><span class="n">_key_fields</span><span class="p">[</span><span class="mi">0</span><span class="p">][</span><span class="mi">1</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">key_type</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key_type</span>
<span class="n">key_exprs</span> <span class="o">=</span> <span class="p">[</span><span class="n">expr</span> <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">expr</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key_fields</span><span class="p">]</span>
<span class="k">return</span> <span class="k">lambda</span> <span class="n">element</span><span class="p">:</span> <span class="n">key_type</span><span class="p">(</span><span class="o">*</span><span class="p">(</span><span class="n">expr</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">key_exprs</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">_key_type_hint</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="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_force_tuple_keys</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_key_fields</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">expr</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key_fields</span><span class="p">[</span><span class="mi">0</span><span class="p">][</span><span class="mi">1</span><span class="p">]</span>
<span class="k">return</span> <span class="n">trivial_inference</span><span class="o">.</span><span class="n">infer_return_type</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="p">[</span><span class="n">input_type</span><span class="p">])</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">row_type</span><span class="o">.</span><span class="n">RowTypeConstraint</span><span class="o">.</span><span class="n">from_fields</span><span class="p">([</span>
<span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">trivial_inference</span><span class="o">.</span><span class="n">infer_return_type</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="p">[</span><span class="n">input_type</span><span class="p">]))</span>
<span class="k">for</span> <span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">expr</span><span class="p">)</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key_fields</span>
<span class="p">])</span>
<div class="viewcode-block" id="GroupBy.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupBy.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;GroupBy(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="s1">&#39;, &#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">name</span> <span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">_</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_key_fields</span><span class="p">)</span></div>
<div class="viewcode-block" id="GroupBy.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.GroupBy.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">input_type</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span> <span class="ow">or</span> <span class="n">typing</span><span class="o">.</span><span class="n">Any</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">pcoll</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="bp">self</span><span class="o">.</span><span class="n">_key_func</span><span class="p">()(</span><span class="n">x</span><span class="p">),</span> <span class="n">x</span><span class="p">))</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_key_type_hint</span><span class="p">(</span><span class="n">input_type</span><span class="p">),</span> <span class="n">input_type</span><span class="p">])</span>
<span class="o">|</span> <span class="n">GroupByKey</span><span class="p">())</span></div></div>
<span class="n">_dynamic_named_tuple_cache</span> <span class="o">=</span> <span class="p">{</span>
<span class="p">}</span> <span class="c1"># type: typing.Dict[typing.Tuple[str, typing.Tuple[str, ...]], typing.Type[tuple]]</span>
<span class="k">def</span> <span class="nf">_dynamic_named_tuple</span><span class="p">(</span><span class="n">type_name</span><span class="p">,</span> <span class="n">field_names</span><span class="p">):</span>
<span class="c1"># type: (str, typing.Tuple[str, ...]) -&gt; typing.Type[tuple]</span>
<span class="n">cache_key</span> <span class="o">=</span> <span class="p">(</span><span class="n">type_name</span><span class="p">,</span> <span class="n">field_names</span><span class="p">)</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">_dynamic_named_tuple_cache</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">cache_key</span><span class="p">)</span>
<span class="k">if</span> <span class="n">result</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="kn">import</span> <span class="nn">collections</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">_dynamic_named_tuple_cache</span><span class="p">[</span><span class="n">cache_key</span><span class="p">]</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">namedtuple</span><span class="p">(</span>
<span class="n">type_name</span><span class="p">,</span> <span class="n">field_names</span><span class="p">)</span>
<span class="c1"># typing: can&#39;t override a method. also, self type is unknown and can&#39;t</span>
<span class="c1"># be cast to tuple</span>
<span class="n">result</span><span class="o">.</span><span class="n">__reduce__</span> <span class="o">=</span> <span class="k">lambda</span> <span class="bp">self</span><span class="p">:</span> <span class="p">(</span> <span class="c1"># type: ignore[assignment]</span>
<span class="n">_unpickle_dynamic_named_tuple</span><span class="p">,</span> <span class="p">(</span><span class="n">type_name</span><span class="p">,</span> <span class="n">field_names</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">(</span><span class="bp">self</span><span class="p">)))</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="k">return</span> <span class="n">result</span>
<span class="k">def</span> <span class="nf">_unpickle_dynamic_named_tuple</span><span class="p">(</span><span class="n">type_name</span><span class="p">,</span> <span class="n">field_names</span><span class="p">,</span> <span class="n">values</span><span class="p">):</span>
<span class="c1"># type: (str, typing.Tuple[str, ...], typing.Iterable[typing.Any]) -&gt; tuple</span>
<span class="k">return</span> <span class="n">_dynamic_named_tuple</span><span class="p">(</span><span class="n">type_name</span><span class="p">,</span> <span class="n">field_names</span><span class="p">)(</span><span class="o">*</span><span class="n">values</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_GroupAndAggregate</span><span class="p">(</span><span class="n">PTransform</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">grouping</span><span class="p">,</span> <span class="n">aggregations</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_grouping</span> <span class="o">=</span> <span class="n">grouping</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_aggregations</span> <span class="o">=</span> <span class="n">aggregations</span>
<span class="k">def</span> <span class="nf">aggregate_field</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">field</span><span class="p">,</span> <span class="c1"># type: typing.Union[str, typing.Callable]</span>
<span class="n">combine_fn</span><span class="p">,</span> <span class="c1"># type: typing.Union[typing.Callable, CombineFn]</span>
<span class="n">dest</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="p">):</span>
<span class="n">field</span> <span class="o">=</span> <span class="n">_expr_to_callable</span><span class="p">(</span><span class="n">field</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_GroupAndAggregate</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_grouping</span><span class="p">,</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_aggregations</span><span class="p">)</span> <span class="o">+</span> <span class="p">[(</span><span class="n">field</span><span class="p">,</span> <span class="n">combine_fn</span><span class="p">,</span> <span class="n">dest</span><span class="p">)])</span>
<span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.combiners</span> <span class="kn">import</span> <span class="n">TupleCombineFn</span>
<span class="n">key_func</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping</span><span class="o">.</span><span class="n">force_tuple_keys</span><span class="p">(</span><span class="kc">True</span><span class="p">)</span><span class="o">.</span><span class="n">_key_func</span><span class="p">()</span>
<span class="n">value_exprs</span> <span class="o">=</span> <span class="p">[</span><span class="n">expr</span> <span class="k">for</span> <span class="n">expr</span><span class="p">,</span> <span class="n">_</span><span class="p">,</span> <span class="n">__</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_aggregations</span><span class="p">]</span>
<span class="n">value_func</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">element</span><span class="p">:</span> <span class="p">[</span><span class="n">expr</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">value_exprs</span><span class="p">]</span>
<span class="n">result_fields</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">name</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">_</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping</span><span class="o">.</span><span class="n">_key_fields</span><span class="p">)</span> <span class="o">+</span> <span class="nb">tuple</span><span class="p">(</span>
<span class="n">dest</span> <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">__</span><span class="p">,</span> <span class="n">dest</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_aggregations</span><span class="p">)</span>
<span class="n">key_type_hint</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping</span><span class="o">.</span><span class="n">force_tuple_keys</span><span class="p">(</span><span class="kc">True</span><span class="p">)</span><span class="o">.</span><span class="n">_key_type_hint</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="k">return</span> <span class="p">(</span>
<span class="n">pcoll</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">key_func</span><span class="p">(</span><span class="n">x</span><span class="p">),</span> <span class="n">value_func</span><span class="p">(</span><span class="n">x</span><span class="p">)))</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span>
<span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">key_type_hint</span><span class="p">,</span> <span class="n">typing</span><span class="o">.</span><span class="n">Any</span><span class="p">])</span>
<span class="o">|</span> <span class="n">CombinePerKey</span><span class="p">(</span>
<span class="n">TupleCombineFn</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span><span class="n">combine_fn</span> <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">combine_fn</span><span class="p">,</span> <span class="n">__</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_aggregations</span><span class="p">]))</span>
<span class="o">|</span> <span class="n">MapTuple</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">key</span><span class="p">,</span>
<span class="n">value</span><span class="p">:</span> <span class="n">_dynamic_named_tuple</span><span class="p">(</span><span class="s1">&#39;Result&#39;</span><span class="p">,</span> <span class="n">result_fields</span><span class="p">)</span>
<span class="p">(</span><span class="o">*</span><span class="p">(</span><span class="n">key</span> <span class="o">+</span> <span class="n">value</span><span class="p">))))</span>
<div class="viewcode-block" id="Select"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Select">[docs]</a><span class="k">class</span> <span class="nc">Select</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Converts the elements of a PCollection into a schema&#39;d PCollection of Rows.</span>
<span class="sd"> `Select(...)` is roughly equivalent to `Map(lambda x: Row(...))` where each</span>
<span class="sd"> argument (which may be a string or callable) of `ToRow` is applied to `x`.</span>
<span class="sd"> For example,</span>
<span class="sd"> pcoll | beam.Select(&#39;a&#39;, b=lambda x: foo(x))</span>
<span class="sd"> is the same as</span>
<span class="sd"> pcoll | beam.Map(lambda x: beam.Row(a=x.a, b=foo(x)))</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="o">*</span><span class="n">args</span><span class="p">,</span> <span class="c1"># type: typing.Union[str, typing.Callable]</span>
<span class="o">**</span><span class="n">kwargs</span> <span class="c1"># type: typing.Union[str, typing.Callable]</span>
<span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_fields</span> <span class="o">=</span> <span class="p">[(</span>
<span class="n">expr</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="k">else</span> <span class="s1">&#39;arg</span><span class="si">%02d</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">ix</span><span class="p">,</span>
<span class="n">_expr_to_callable</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span> <span class="k">for</span> <span class="p">(</span><span class="n">ix</span><span class="p">,</span> <span class="n">expr</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">args</span><span class="p">)</span>
<span class="p">]</span> <span class="o">+</span> <span class="p">[(</span><span class="n">name</span><span class="p">,</span> <span class="n">_expr_to_callable</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">name</span><span class="p">))</span>
<span class="k">for</span> <span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">expr</span><span class="p">)</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">()]</span>
<div class="viewcode-block" id="Select.default_label"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Select.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;ToRows(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="s1">&#39;, &#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">name</span> <span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">_</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fields</span><span class="p">)</span></div>
<div class="viewcode-block" id="Select.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Select.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">Map</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">Row</span><span class="p">(</span><span class="o">**</span><span class="p">{</span><span class="n">name</span><span class="p">:</span> <span class="n">expr</span><span class="p">(</span><span class="n">x</span><span class="p">)</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">expr</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fields</span><span class="p">}))</span></div>
<div class="viewcode-block" id="Select.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Select.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_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">return</span> <span class="n">row_type</span><span class="o">.</span><span class="n">RowTypeConstraint</span><span class="o">.</span><span class="n">from_fields</span><span class="p">([</span>
<span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">trivial_inference</span><span class="o">.</span><span class="n">infer_return_type</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="p">[</span><span class="n">input_type</span><span class="p">]))</span>
<span class="k">for</span> <span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">expr</span><span class="p">)</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fields</span>
<span class="p">])</span></div></div>
<div class="viewcode-block" id="Partition"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Partition">[docs]</a><span class="k">class</span> <span class="nc">Partition</span><span class="p">(</span><span class="n">PTransformWithSideInputs</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Split a PCollection into several partitions.</span>
<span class="sd"> Uses the specified PartitionFn to separate an input PCollection into the</span>
<span class="sd"> specified number of sub-PCollections.</span>
<span class="sd"> When apply()d, a Partition() PTransform requires the following:</span>
<span class="sd"> Args:</span>
<span class="sd"> partitionfn: a PartitionFn, or a callable with the signature described in</span>
<span class="sd"> CallableWrapperPartitionFn.</span>
<span class="sd"> n: number of output partitions.</span>
<span class="sd"> The result of this PTransform is a simple list of the output PCollections</span>
<span class="sd"> representing each of n partitions, in order.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="Partition.ApplyPartitionFnFn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Partition.ApplyPartitionFnFn">[docs]</a> <span class="k">class</span> <span class="nc">ApplyPartitionFnFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A DoFn that applies a PartitionFn.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Partition.ApplyPartitionFnFn.process"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Partition.ApplyPartitionFnFn.process">[docs]</a> <span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">partitionfn</span><span class="p">,</span> <span class="n">n</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">partition</span> <span class="o">=</span> <span class="n">partitionfn</span><span class="o">.</span><span class="n">partition_for</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">n</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">if</span> <span class="ow">not</span> <span class="mi">0</span> <span class="o">&lt;=</span> <span class="n">partition</span> <span class="o">&lt;</span> <span class="n">n</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;PartitionFn specified out-of-bounds partition index: &#39;</span>
<span class="s1">&#39;</span><span class="si">%d</span><span class="s1"> not in [0, </span><span class="si">%d</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">partition</span><span class="p">,</span> <span class="n">n</span><span class="p">))</span>
<span class="c1"># Each input is directed into the output that corresponds to the</span>
<span class="c1"># selected partition.</span>
<span class="k">yield</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">TaggedOutput</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">partition</span><span class="p">),</span> <span class="n">element</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="Partition.make_fn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Partition.make_fn">[docs]</a> <span class="k">def</span> <span class="nf">make_fn</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">has_side_inputs</span><span class="p">):</span>
<span class="k">return</span> <span class="n">fn</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">PartitionFn</span><span class="p">)</span> <span class="k">else</span> <span class="n">CallableWrapperPartitionFn</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span></div>
<div class="viewcode-block" id="Partition.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Partition.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">n</span> <span class="o">=</span> <span class="nb">int</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span>
<span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span> <span class="o">=</span> <span class="n">util</span><span class="o">.</span><span class="n">insert_values_in_args</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">kwargs</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">ApplyPartitionFnFn</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">fn</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="o">.</span><span class="n">with_outputs</span><span class="p">(</span><span class="o">*</span><span class="p">[</span><span class="nb">str</span><span class="p">(</span><span class="n">t</span><span class="p">)</span> <span class="k">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">n</span><span class="p">)])</span></div></div>
<div class="viewcode-block" id="Windowing"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Windowing">[docs]</a><span class="k">class</span> <span class="nc">Windowing</span><span class="p">(</span><span class="nb">object</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">windowfn</span><span class="p">,</span> <span class="c1"># type: WindowFn</span>
<span class="n">triggerfn</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: typing.Optional[TriggerFn]</span>
<span class="n">accumulation_mode</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: typing.Optional[beam_runner_api_pb2.AccumulationMode.Enum.ValueType]</span>
<span class="n">timestamp_combiner</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: typing.Optional[beam_runner_api_pb2.OutputTime.Enum.ValueType]</span>
<span class="n">allowed_lateness</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="c1"># type: typing.Union[int, float]</span>
<span class="n">environment_id</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: typing.Optional[str]</span>
<span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Class representing the window strategy.</span>
<span class="sd"> Args:</span>
<span class="sd"> windowfn: Window assign function.</span>
<span class="sd"> triggerfn: Trigger function.</span>
<span class="sd"> accumulation_mode: a AccumulationMode, controls what to do with data</span>
<span class="sd"> when a trigger fires multiple times.</span>
<span class="sd"> timestamp_combiner: a TimestampCombiner, determines how output</span>
<span class="sd"> timestamps of grouping operations are assigned.</span>
<span class="sd"> allowed_lateness: Maximum delay in seconds after end of window</span>
<span class="sd"> allowed for any late data to be processed without being discarded</span>
<span class="sd"> directly.</span>
<span class="sd"> environment_id: Environment where the current window_fn should be</span>
<span class="sd"> applied in.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">global</span> <span class="n">AccumulationMode</span><span class="p">,</span> <span class="n">DefaultTrigger</span> <span class="c1"># pylint: disable=global-variable-not-assigned</span>
<span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">AccumulationMode</span><span class="p">,</span> <span class="n">DefaultTrigger</span>
<span class="c1"># pylint: enable=wrong-import-order, wrong-import-position</span>
<span class="k">if</span> <span class="n">triggerfn</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">triggerfn</span> <span class="o">=</span> <span class="n">DefaultTrigger</span><span class="p">()</span>
<span class="k">if</span> <span class="n">accumulation_mode</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">triggerfn</span> <span class="o">==</span> <span class="n">DefaultTrigger</span><span class="p">():</span>
<span class="n">accumulation_mode</span> <span class="o">=</span> <span class="n">AccumulationMode</span><span class="o">.</span><span class="n">DISCARDING</span>
<span class="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;accumulation_mode must be provided for non-trivial triggers&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">windowfn</span><span class="o">.</span><span class="n">get_window_coder</span><span class="p">()</span><span class="o">.</span><span class="n">is_deterministic</span><span class="p">():</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;window fn (</span><span class="si">%s</span><span class="s1">) does not have a determanistic coder (</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span>
<span class="p">(</span><span class="n">windowfn</span><span class="p">,</span> <span class="n">windowfn</span><span class="o">.</span><span class="n">get_window_coder</span><span class="p">()))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">windowfn</span> <span class="o">=</span> <span class="n">windowfn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggerfn</span> <span class="o">=</span> <span class="n">triggerfn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">accumulation_mode</span> <span class="o">=</span> <span class="n">accumulation_mode</span>
<span class="bp">self</span><span class="o">.</span><span class="n">allowed_lateness</span> <span class="o">=</span> <span class="n">Duration</span><span class="o">.</span><span class="n">of</span><span class="p">(</span><span class="n">allowed_lateness</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">environment_id</span> <span class="o">=</span> <span class="n">environment_id</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">timestamp_combiner</span> <span class="ow">or</span> <span class="n">TimestampCombiner</span><span class="o">.</span><span class="n">OUTPUT_AT_EOW</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_is_default</span> <span class="o">=</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">windowfn</span> <span class="o">==</span> <span class="n">GlobalWindows</span><span class="p">()</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggerfn</span> <span class="o">==</span> <span class="n">DefaultTrigger</span><span class="p">()</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">accumulation_mode</span> <span class="o">==</span> <span class="n">AccumulationMode</span><span class="o">.</span><span class="n">DISCARDING</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner</span> <span class="o">==</span> <span class="n">TimestampCombiner</span><span class="o">.</span><span class="n">OUTPUT_AT_EOW</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">allowed_lateness</span> <span class="o">==</span> <span class="mi">0</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s2">&quot;Windowing(</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">, </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="n">windowfn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggerfn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">accumulation_mode</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">environment_id</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__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">type</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">==</span> <span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_default</span> <span class="ow">and</span> <span class="n">other</span><span class="o">.</span><span class="n">_is_default</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">return</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">windowfn</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">windowfn</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggerfn</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">triggerfn</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">accumulation_mode</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">accumulation_mode</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">timestamp_combiner</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">allowed_lateness</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">allowed_lateness</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">environment_id</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">environment_id</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">def</span> <span class="fm">__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">windowfn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggerfn</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">accumulation_mode</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">allowed_lateness</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">environment_id</span><span class="p">))</span>
<div class="viewcode-block" id="Windowing.is_default"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Windowing.is_default">[docs]</a> <span class="k">def</span> <span class="nf">is_default</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">_is_default</span></div>
<div class="viewcode-block" id="Windowing.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Windowing.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="c1"># type: (PipelineContext) -&gt; beam_runner_api_pb2.WindowingStrategy</span>
<span class="n">environment_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">environment_id</span> <span class="ow">or</span> <span class="n">context</span><span class="o">.</span><span class="n">default_environment_id</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">WindowingStrategy</span><span class="p">(</span>
<span class="n">window_fn</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">windowfn</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="c1"># TODO(robertwb): Prohibit implicit multi-level merging.</span>
<span class="n">merge_status</span><span class="o">=</span><span class="p">(</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">MergeStatus</span><span class="o">.</span><span class="n">NEEDS_MERGE</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">windowfn</span><span class="o">.</span><span class="n">is_merging</span><span class="p">()</span> <span class="k">else</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">MergeStatus</span><span class="o">.</span><span class="n">NON_MERGING</span><span class="p">),</span>
<span class="n">window_coder_id</span><span class="o">=</span><span class="n">context</span><span class="o">.</span><span class="n">coders</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">windowfn</span><span class="o">.</span><span class="n">get_window_coder</span><span class="p">()),</span>
<span class="n">trigger</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">triggerfn</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="n">accumulation_mode</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">accumulation_mode</span><span class="p">,</span>
<span class="n">output_time</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner</span><span class="p">,</span>
<span class="c1"># TODO(robertwb): Support EMIT_IF_NONEMPTY</span>
<span class="n">closing_behavior</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ClosingBehavior</span><span class="o">.</span><span class="n">EMIT_ALWAYS</span><span class="p">,</span>
<span class="n">on_time_behavior</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">OnTimeBehavior</span><span class="o">.</span><span class="n">FIRE_ALWAYS</span><span class="p">,</span>
<span class="n">allowed_lateness</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">allowed_lateness</span><span class="o">.</span><span class="n">micros</span> <span class="o">//</span> <span class="mi">1000</span><span class="p">,</span>
<span class="n">environment_id</span><span class="o">=</span><span class="n">environment_id</span><span class="p">)</span></div>
<div class="viewcode-block" id="Windowing.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Windowing.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"># pylint: disable=wrong-import-order, wrong-import-position</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">TriggerFn</span>
<span class="k">return</span> <span class="n">Windowing</span><span class="p">(</span>
<span class="n">windowfn</span><span class="o">=</span><span class="n">WindowFn</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">window_fn</span><span class="p">,</span> <span class="n">context</span><span class="p">),</span>
<span class="n">triggerfn</span><span class="o">=</span><span class="n">TriggerFn</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">trigger</span><span class="p">,</span> <span class="n">context</span><span class="p">),</span>
<span class="n">accumulation_mode</span><span class="o">=</span><span class="n">proto</span><span class="o">.</span><span class="n">accumulation_mode</span><span class="p">,</span>
<span class="n">timestamp_combiner</span><span class="o">=</span><span class="n">proto</span><span class="o">.</span><span class="n">output_time</span><span class="p">,</span>
<span class="n">allowed_lateness</span><span class="o">=</span><span class="n">Duration</span><span class="p">(</span><span class="n">micros</span><span class="o">=</span><span class="n">proto</span><span class="o">.</span><span class="n">allowed_lateness</span> <span class="o">*</span> <span class="mi">1000</span><span class="p">),</span>
<span class="n">environment_id</span><span class="o">=</span><span class="kc">None</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="WindowInto"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WindowInto">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">WindowInto</span><span class="p">(</span><span class="n">ParDo</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A window transform assigning windows to each element of a PCollection.</span>
<span class="sd"> Transforms an input PCollection by applying a windowing function to each</span>
<span class="sd"> element. Each transformed element in the result will be a WindowedValue</span>
<span class="sd"> element with the same input value and timestamp, with its new set of windows</span>
<span class="sd"> determined by the windowing function.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="WindowInto.WindowIntoFn"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WindowInto.WindowIntoFn">[docs]</a> <span class="k">class</span> <span class="nc">WindowIntoFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A DoFn that applies a WindowInto operation.&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">windowing</span><span class="p">):</span>
<span class="c1"># type: (Windowing) -&gt; None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">windowing</span> <span class="o">=</span> <span class="n">windowing</span>
<div class="viewcode-block" id="WindowInto.WindowIntoFn.process"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WindowInto.WindowIntoFn.process">[docs]</a> <span class="k">def</span> <span class="nf">process</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">):</span>
<span class="n">context</span> <span class="o">=</span> <span class="n">WindowFn</span><span class="o">.</span><span class="n">AssignContext</span><span class="p">(</span>
<span class="n">timestamp</span><span class="p">,</span> <span class="n">element</span><span class="o">=</span><span class="n">element</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">window</span><span class="p">)</span>
<span class="n">new_windows</span> <span class="o">=</span> <span class="bp">self</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">assign</span><span class="p">(</span><span class="n">context</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">WindowedValue</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">context</span><span class="o">.</span><span class="n">timestamp</span><span class="p">,</span> <span class="n">new_windows</span><span class="p">)</span></div></div>
<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">windowfn</span><span class="p">,</span> <span class="c1"># type: typing.Union[Windowing, WindowFn]</span>
<span class="n">trigger</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: typing.Optional[TriggerFn]</span>
<span class="n">accumulation_mode</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">timestamp_combiner</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">allowed_lateness</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Initializes a WindowInto transform.</span>
<span class="sd"> Args:</span>
<span class="sd"> windowfn (Windowing, WindowFn): Function to be used for windowing.</span>
<span class="sd"> trigger: (optional) Trigger used for windowing, or None for default.</span>
<span class="sd"> accumulation_mode: (optional) Accumulation mode used for windowing,</span>
<span class="sd"> required for non-trivial triggers.</span>
<span class="sd"> timestamp_combiner: (optional) Timestamp combniner used for windowing,</span>
<span class="sd"> or None for default.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">windowfn</span><span class="p">,</span> <span class="n">Windowing</span><span class="p">):</span>
<span class="c1"># Overlay windowing with kwargs.</span>
<span class="n">windowing</span> <span class="o">=</span> <span class="n">windowfn</span>
<span class="n">windowfn</span> <span class="o">=</span> <span class="n">windowing</span><span class="o">.</span><span class="n">windowfn</span>
<span class="c1"># Use windowing to fill in defaults for the extra arguments.</span>
<span class="n">trigger</span> <span class="o">=</span> <span class="n">trigger</span> <span class="ow">or</span> <span class="n">windowing</span><span class="o">.</span><span class="n">triggerfn</span>
<span class="n">accumulation_mode</span> <span class="o">=</span> <span class="n">accumulation_mode</span> <span class="ow">or</span> <span class="n">windowing</span><span class="o">.</span><span class="n">accumulation_mode</span>
<span class="n">timestamp_combiner</span> <span class="o">=</span> <span class="n">timestamp_combiner</span> <span class="ow">or</span> <span class="n">windowing</span><span class="o">.</span><span class="n">timestamp_combiner</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="p">(</span>
<span class="n">windowfn</span><span class="p">,</span>
<span class="n">trigger</span><span class="p">,</span>
<span class="n">accumulation_mode</span><span class="p">,</span>
<span class="n">timestamp_combiner</span><span class="p">,</span>
<span class="n">allowed_lateness</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">WindowIntoFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">windowing</span><span class="p">))</span>
<div class="viewcode-block" id="WindowInto.get_windowing"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WindowInto.get_windowing">[docs]</a> <span class="k">def</span> <span class="nf">get_windowing</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_inputs</span><span class="p">):</span>
<span class="c1"># type: (typing.Any) -&gt; Windowing</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">windowing</span></div>
<div class="viewcode-block" id="WindowInto.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WindowInto.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_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">return</span> <span class="n">input_type</span></div>
<div class="viewcode-block" id="WindowInto.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WindowInto.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">input_type</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span>
<span class="k">if</span> <span class="n">input_type</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">output_type</span> <span class="o">=</span> <span class="n">input_type</span>
<span class="bp">self</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">input_type</span><span class="p">)</span>
<span class="bp">self</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>
<span class="k">return</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">expand</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span></div>
<span class="c1"># typing: PTransform base class does not accept extra_kwargs</span>
<div class="viewcode-block" id="WindowInto.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WindowInto.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</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="o">**</span><span class="n">extra_kwargs</span><span class="p">):</span> <span class="c1"># type: ignore[override]</span>
<span class="c1"># type: (PipelineContext, **typing.Any) -&gt; typing.Tuple[str, message.Message]</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">ASSIGN_WINDOWS</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">windowing</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></div>
<div class="viewcode-block" id="WindowInto.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.WindowInto.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_ptransform</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="n">windowing</span> <span class="o">=</span> <span class="n">Windowing</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">return</span> <span class="n">WindowInto</span><span class="p">(</span>
<span class="n">windowing</span><span class="o">.</span><span class="n">windowfn</span><span class="p">,</span>
<span class="n">trigger</span><span class="o">=</span><span class="n">windowing</span><span class="o">.</span><span class="n">triggerfn</span><span class="p">,</span>
<span class="n">accumulation_mode</span><span class="o">=</span><span class="n">windowing</span><span class="o">.</span><span class="n">accumulation_mode</span><span class="p">,</span>
<span class="n">timestamp_combiner</span><span class="o">=</span><span class="n">windowing</span><span class="o">.</span><span class="n">timestamp_combiner</span><span class="p">)</span></div></div>
<span class="n">PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">ASSIGN_WINDOWS</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="c1"># TODO(robertwb): Update WindowIntoPayload to include the full strategy.</span>
<span class="c1"># (Right now only WindowFn is used, but we need this to reconstitute the</span>
<span class="c1"># WindowInto transform, and in the future will need it at runtime to</span>
<span class="c1"># support meta-data driven triggers.)</span>
<span class="c1"># TODO(robertwb): Use a reference rather than embedding?</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">WindowingStrategy</span><span class="p">,</span>
<span class="n">WindowInto</span><span class="o">.</span><span class="n">from_runner_api_parameter</span><span class="p">)</span>
<span class="c1"># Python&#39;s pickling is broken for nested classes.</span>
<span class="n">WindowIntoFn</span> <span class="o">=</span> <span class="n">WindowInto</span><span class="o">.</span><span class="n">WindowIntoFn</span>
<div class="viewcode-block" id="Flatten"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Flatten">[docs]</a><span class="k">class</span> <span class="nc">Flatten</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Merges several PCollections into a single PCollection.</span>
<span class="sd"> Copies all elements in 0 or more PCollections into a single output</span>
<span class="sd"> PCollection. If there are no input PCollections, the resulting PCollection</span>
<span class="sd"> will be empty (but see also kwargs below).</span>
<span class="sd"> Args:</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 Flatten can obtain this</span>
<span class="sd"> information from one of the input PCollections, but if there are none (or</span>
<span class="sd"> if there&#39;s a chance there may be none), this argument is the only way to</span>
<span class="sd"> provide pipeline information and should be considered mandatory.</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="o">**</span><span class="n">kwargs</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">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="c1"># type: typing.Optional[Pipeline]</span>
<span class="k">if</span> <span class="n">kwargs</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Unexpected keyword arguments: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="nb">list</span><span class="p">(</span><span class="n">kwargs</span><span class="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="n">pvalueish</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">except</span> <span class="ne">TypeError</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Input to Flatten must be an iterable. &#39;</span>
<span class="s1">&#39;Got a value of type </span><span class="si">%s</span><span class="s1"> instead.&#39;</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">pvalueish</span><span class="p">))</span>
<span class="k">return</span> <span class="n">pvalueish</span><span class="p">,</span> <span class="n">pvalueish</span>
<div class="viewcode-block" id="Flatten.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Flatten.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="k">for</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="n">is_bounded</span> <span class="o">=</span> <span class="nb">all</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">is_bounded</span> <span class="k">for</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pvalue</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">is_bounded</span><span class="o">=</span><span class="n">is_bounded</span><span class="p">)</span></div>
<div class="viewcode-block" id="Flatten.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Flatten.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_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">return</span> <span class="n">input_type</span></div>
<div class="viewcode-block" id="Flatten.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Flatten.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</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="c1"># type: (PipelineContext) -&gt; typing.Tuple[str, None]</span>
<span class="k">return</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">FLATTEN</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span></div>
<div class="viewcode-block" id="Flatten.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Flatten.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span>
<span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">unused_parameter</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">Flatten</span><span class="p">()</span></div></div>
<span class="n">PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">FLATTEN</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span><span class="p">,</span> <span class="n">Flatten</span><span class="o">.</span><span class="n">from_runner_api_parameter</span><span class="p">)</span>
<div class="viewcode-block" id="Create"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Create">[docs]</a><span class="k">class</span> <span class="nc">Create</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A transform that creates a PCollection from an iterable.&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">values</span><span class="p">,</span> <span class="n">reshuffle</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Initializes a Create transform.</span>
<span class="sd"> Args:</span>
<span class="sd"> values: An object of values for the PCollection</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="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="p">(</span><span class="nb">str</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;PTransform Create: Refusing to treat string as &#39;</span>
<span class="s1">&#39;an iterable. (string=</span><span class="si">%r</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">values</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="n">values</span> <span class="o">=</span> <span class="n">values</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">values</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">values</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">reshuffle</span> <span class="o">=</span> <span class="n">reshuffle</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_coder</span> <span class="o">=</span> <span class="n">typecoders</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">get_output_type</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">__getstate__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">serialized_values</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">values</span><span class="p">]</span>
<span class="k">return</span> <span class="n">serialized_values</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">reshuffle</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span>
<span class="k">def</span> <span class="nf">__setstate__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">state</span><span class="p">):</span>
<span class="n">serialized_values</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">reshuffle</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span> <span class="o">=</span> <span class="n">state</span>
<span class="bp">self</span><span class="o">.</span><span class="n">values</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">decode</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">serialized_values</span><span class="p">]</span>
<div class="viewcode-block" id="Create.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Create.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</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="c1"># type: (PipelineContext) -&gt; typing.Tuple[str, bytes]</span>
<span class="c1"># Required as this is identified by type in PTransformOverrides.</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/18713): Use an actual URN</span>
<span class="c1"># here.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_runner_api_pickled</span><span class="p">(</span><span class="n">context</span><span class="p">)</span></div>
<div class="viewcode-block" id="Create.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Create.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_type</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_input_type</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">values</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="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Union</span><span class="p">[[</span>
<span class="n">trivial_inference</span><span class="o">.</span><span class="n">instance_to_type</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">values</span>
<span class="p">]]</span></div>
<div class="viewcode-block" id="Create.get_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Create.get_output_type">[docs]</a> <span class="k">def</span> <span class="nf">get_output_type</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="bp">self</span><span class="o">.</span><span class="n">get_type_hints</span><span class="p">()</span><span class="o">.</span><span class="n">simple_output_type</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">label</span><span class="p">)</span> <span class="ow">or</span>
<span class="bp">self</span><span class="o">.</span><span class="n">infer_output_type</span><span class="p">(</span><span class="kc">None</span><span class="p">))</span></div>
<div class="viewcode-block" id="Create.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Create.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">pbegin</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pbegin</span><span class="p">,</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PBegin</span><span class="p">)</span>
<span class="n">serialized_values</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">values</span><span class="p">]</span>
<span class="n">reshuffle</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">reshuffle</span>
<span class="c1"># Avoid the &quot;redistributing&quot; reshuffle for 0 and 1 element Creates.</span>
<span class="c1"># These special cases are often used in building up more complex</span>
<span class="c1"># transforms (e.g. Write).</span>
<span class="k">class</span> <span class="nc">MaybeReshuffle</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">serialized_values</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</span> <span class="ow">and</span> <span class="n">reshuffle</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.util</span> <span class="kn">import</span> <span class="n">Reshuffle</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Reshuffle</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="k">return</span> <span class="p">(</span>
<span class="n">pbegin</span>
<span class="o">|</span> <span class="n">Impulse</span><span class="p">()</span>
<span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="k">lambda</span> <span class="n">_</span><span class="p">:</span> <span class="n">serialized_values</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="nb">bytes</span><span class="p">)</span>
<span class="o">|</span> <span class="n">MaybeReshuffle</span><span class="p">()</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="nb">bytes</span><span class="p">)</span>
<span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">decode</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">get_output_type</span><span class="p">()))</span></div>
<div class="viewcode-block" id="Create.as_read"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Create.as_read">[docs]</a> <span class="k">def</span> <span class="nf">as_read</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="kn">from</span> <span class="nn">apache_beam.io</span> <span class="kn">import</span> <span class="n">iobase</span>
<span class="n">source</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_create_source_from_iterable</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">values</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="p">)</span>
<span class="k">return</span> <span class="n">iobase</span><span class="o">.</span><span class="n">Read</span><span class="p">(</span><span class="n">source</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">get_output_type</span><span class="p">())</span></div>
<div class="viewcode-block" id="Create.get_windowing"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Create.get_windowing">[docs]</a> <span class="k">def</span> <span class="nf">get_windowing</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_inputs</span><span class="p">):</span>
<span class="c1"># type: (typing.Any) -&gt; Windowing</span>
<span class="k">return</span> <span class="n">Windowing</span><span class="p">(</span><span class="n">GlobalWindows</span><span class="p">())</span></div>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_create_source_from_iterable</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="n">coder</span><span class="p">):</span>
<span class="k">return</span> <span class="n">Create</span><span class="o">.</span><span class="n">_create_source</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="n">coder</span><span class="o">.</span><span class="n">encode</span><span class="p">,</span> <span class="n">values</span><span class="p">)),</span> <span class="n">coder</span><span class="p">)</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_create_source</span><span class="p">(</span><span class="n">serialized_values</span><span class="p">,</span> <span class="n">coder</span><span class="p">):</span>
<span class="c1"># type: (typing.Any, typing.Any) -&gt; create_source._CreateSource</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.create_source</span> <span class="kn">import</span> <span class="n">_CreateSource</span>
<span class="k">return</span> <span class="n">_CreateSource</span><span class="p">(</span><span class="n">serialized_values</span><span class="p">,</span> <span class="n">coder</span><span class="p">)</span></div>
<div class="viewcode-block" id="Impulse"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Impulse">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="nb">bytes</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">Impulse</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Impulse primitive.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Impulse.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Impulse.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">pbegin</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">pbegin</span><span class="p">,</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PBegin</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;Input to Impulse transform must be a PBegin but found </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">pbegin</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">(</span><span class="n">pbegin</span><span class="o">.</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">element_type</span><span class="o">=</span><span class="nb">bytes</span><span class="p">)</span></div>
<div class="viewcode-block" id="Impulse.get_windowing"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Impulse.get_windowing">[docs]</a> <span class="k">def</span> <span class="nf">get_windowing</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">inputs</span><span class="p">):</span>
<span class="c1"># type: (typing.Any) -&gt; Windowing</span>
<span class="k">return</span> <span class="n">Windowing</span><span class="p">(</span><span class="n">GlobalWindows</span><span class="p">())</span></div>
<div class="viewcode-block" id="Impulse.infer_output_type"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Impulse.infer_output_type">[docs]</a> <span class="k">def</span> <span class="nf">infer_output_type</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_input_type</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">bytes</span></div>
<div class="viewcode-block" id="Impulse.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Impulse.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; typing.Tuple[str, None]</span>
<span class="k">return</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">IMPULSE</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span></div>
<div class="viewcode-block" id="Impulse.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.core.html#apache_beam.transforms.combinefn_lifecycle_pipeline.Impulse.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">IMPULSE</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span>
<span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">unused_parameter</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">Impulse</span><span class="p">()</span></div></div>
<span class="k">def</span> <span class="nf">_strip_output_annotations</span><span class="p">(</span><span class="n">type_hint</span><span class="p">):</span>
<span class="c1"># TODO(robertwb): These should be parameterized types that the</span>
<span class="c1"># type inferencer understands.</span>
<span class="c1"># Then we can replace them with the correct element types instead of</span>
<span class="c1"># using Any. Refer to typehints.WindowedValue when doing this.</span>
<span class="n">annotations</span> <span class="o">=</span> <span class="p">(</span><span class="n">TimestampedValue</span><span class="p">,</span> <span class="n">WindowedValue</span><span class="p">,</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">TaggedOutput</span><span class="p">)</span>
<span class="n">contains_annotation</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">def</span> <span class="nf">visitor</span><span class="p">(</span><span class="n">t</span><span class="p">,</span> <span class="n">unused_args</span><span class="p">):</span>
<span class="k">if</span> <span class="n">t</span> <span class="ow">in</span> <span class="n">annotations</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">StopIteration</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">visit_inner_types</span><span class="p">(</span><span class="n">type_hint</span><span class="p">,</span> <span class="n">visitor</span><span class="p">,</span> <span class="p">[])</span>
<span class="k">except</span> <span class="ne">StopIteration</span><span class="p">:</span>
<span class="n">contains_annotation</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span> <span class="k">if</span> <span class="n">contains_annotation</span> <span class="k">else</span> <span class="n">type_hint</span>
</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>