blob: c2c94d6365cb4a0676578b0789fc3dfb946116ca [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.runners.direct.sdf_direct_runner &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.runners.direct.sdf_direct_runner</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.runners.direct.sdf_direct_runner</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;This module contains Splittable DoFn logic that is specific to DirectRunner.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span> <span class="nn">uuid</span>
<span class="kn">from</span> <span class="nn">threading</span> <span class="kn">import</span> <span class="n">Lock</span>
<span class="kn">from</span> <span class="nn">threading</span> <span class="kn">import</span> <span class="n">Timer</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TYPE_CHECKING</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Iterable</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Optional</span>
<span class="kn">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">TimeDomain</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.coders</span> <span class="kn">import</span> <span class="n">typecoders</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="kn">import</span> <span class="n">AppliedPTransform</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="kn">import</span> <span class="n">PTransformOverride</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.common</span> <span class="kn">import</span> <span class="n">DoFnContext</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.common</span> <span class="kn">import</span> <span class="n">DoFnInvoker</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="kn">from</span> <span class="nn">apache_beam.runners.common</span> <span class="kn">import</span> <span class="n">OutputHandler</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.evaluation_context</span> <span class="kn">import</span> <span class="n">DirectStepContext</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.util</span> <span class="kn">import</span> <span class="n">KeyedWorkItem</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.watermark_manager</span> <span class="kn">import</span> <span class="n">WatermarkManager</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">ParDo</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">ProcessContinuation</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.trigger</span> <span class="kn">import</span> <span class="n">_ReadModifyWriteStateTag</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.windowed_value</span> <span class="kn">import</span> <span class="n">WindowedValue</span>
<span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">apache_beam.iobase</span> <span class="kn">import</span> <span class="n">WatermarkEstimator</span>
<div class="viewcode-block" id="SplittableParDoOverride"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SplittableParDoOverride">[docs]</a><span class="k">class</span> <span class="nc">SplittableParDoOverride</span><span class="p">(</span><span class="n">PTransformOverride</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A transform override for ParDo transformss of SplittableDoFns.</span>
<span class="sd"> Replaces the ParDo transform with a SplittableParDo transform that performs</span>
<span class="sd"> SDF specific logic.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="SplittableParDoOverride.matches"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SplittableParDoOverride.matches">[docs]</a> <span class="k">def</span> <span class="nf">matches</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="p">,</span> <span class="n">AppliedPTransform</span><span class="p">)</span>
<span class="n">transform</span> <span class="o">=</span> <span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">ParDo</span><span class="p">):</span>
<span class="n">signature</span> <span class="o">=</span> <span class="n">DoFnSignature</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">fn</span><span class="p">)</span>
<span class="k">return</span> <span class="n">signature</span><span class="o">.</span><span class="n">is_splittable_dofn</span><span class="p">()</span></div>
<div class="viewcode-block" id="SplittableParDoOverride.get_replacement_transform_for_applied_ptransform"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SplittableParDoOverride.get_replacement_transform_for_applied_ptransform">[docs]</a> <span class="k">def</span> <span class="nf">get_replacement_transform_for_applied_ptransform</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="n">ptransform</span> <span class="o">=</span> <span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ptransform</span><span class="p">,</span> <span class="n">ParDo</span><span class="p">)</span>
<span class="n">do_fn</span> <span class="o">=</span> <span class="n">ptransform</span><span class="o">.</span><span class="n">fn</span>
<span class="n">signature</span> <span class="o">=</span> <span class="n">DoFnSignature</span><span class="p">(</span><span class="n">do_fn</span><span class="p">)</span>
<span class="k">if</span> <span class="n">signature</span><span class="o">.</span><span class="n">is_splittable_dofn</span><span class="p">():</span>
<span class="k">return</span> <span class="n">SplittableParDo</span><span class="p">(</span><span class="n">ptransform</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">ptransform</span></div></div>
<div class="viewcode-block" id="SplittableParDo"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SplittableParDo">[docs]</a><span class="k">class</span> <span class="nc">SplittableParDo</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 processes a PCollection using a Splittable DoFn.&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">ptransform</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ptransform</span><span class="p">,</span> <span class="n">ParDo</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_ptransform</span> <span class="o">=</span> <span class="n">ptransform</span>
<div class="viewcode-block" id="SplittableParDo.expand"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SplittableParDo.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">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ptransform</span><span class="o">.</span><span class="n">fn</span>
<span class="n">signature</span> <span class="o">=</span> <span class="n">DoFnSignature</span><span class="p">(</span><span class="n">sdf</span><span class="p">)</span>
<span class="n">restriction_coder</span> <span class="o">=</span> <span class="n">signature</span><span class="o">.</span><span class="n">get_restriction_coder</span><span class="p">()</span>
<span class="n">element_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="n">pcoll</span><span class="o">.</span><span class="n">element_type</span><span class="p">)</span>
<span class="n">keyed_elements</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">pcoll</span>
<span class="o">|</span> <span class="s1">&#39;pair&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">PairWithRestrictionFn</span><span class="p">(</span><span class="n">sdf</span><span class="p">))</span>
<span class="o">|</span> <span class="s1">&#39;split&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">SplitRestrictionFn</span><span class="p">(</span><span class="n">sdf</span><span class="p">))</span>
<span class="o">|</span> <span class="s1">&#39;explode&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">ExplodeWindowsFn</span><span class="p">())</span>
<span class="o">|</span> <span class="s1">&#39;random&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">ParDo</span><span class="p">(</span><span class="n">RandomUniqueKeyFn</span><span class="p">()))</span>
<span class="k">return</span> <span class="n">keyed_elements</span> <span class="o">|</span> <span class="n">ProcessKeyedElements</span><span class="p">(</span>
<span class="n">sdf</span><span class="p">,</span>
<span class="n">element_coder</span><span class="p">,</span>
<span class="n">restriction_coder</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="bp">self</span><span class="o">.</span><span class="n">_ptransform</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">_ptransform</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">_ptransform</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="ElementAndRestriction"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ElementAndRestriction">[docs]</a><span class="k">class</span> <span class="nc">ElementAndRestriction</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 holder for an element, restriction, and watermark estimator state.&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">element</span><span class="p">,</span> <span class="n">restriction</span><span class="p">,</span> <span class="n">watermark_estimator_state</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">element</span>
<span class="bp">self</span><span class="o">.</span><span class="n">restriction</span> <span class="o">=</span> <span class="n">restriction</span>
<span class="bp">self</span><span class="o">.</span><span class="n">watermark_estimator_state</span> <span class="o">=</span> <span class="n">watermark_estimator_state</span></div>
<div class="viewcode-block" id="PairWithRestrictionFn"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.PairWithRestrictionFn">[docs]</a><span class="k">class</span> <span class="nc">PairWithRestrictionFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A transform that pairs each element with a restriction.&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">do_fn</span><span class="p">):</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="n">do_fn</span><span class="p">)</span>
<div class="viewcode-block" id="PairWithRestrictionFn.start_bundle"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.PairWithRestrictionFn.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="bp">self</span><span class="o">.</span><span class="n">_invoker</span> <span class="o">=</span> <span class="n">DoFnInvoker</span><span class="o">.</span><span class="n">create_invoker</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_signature</span><span class="p">,</span>
<span class="n">output_processor</span><span class="o">=</span><span class="n">_NoneShallPassOutputHandler</span><span class="p">(),</span>
<span class="n">process_invocation</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span></div>
<div class="viewcode-block" id="PairWithRestrictionFn.process"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.PairWithRestrictionFn.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">beam</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="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">initial_restriction</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_invoker</span><span class="o">.</span><span class="n">invoke_initial_restriction</span><span class="p">(</span><span class="n">element</span><span class="p">)</span>
<span class="n">watermark_estimator_state</span> <span class="o">=</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_signature</span><span class="o">.</span><span class="n">process_method</span><span class="o">.</span><span class="n">watermark_estimator_provider</span><span class="o">.</span>
<span class="n">initial_estimator_state</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">initial_restriction</span><span class="p">))</span>
<span class="k">yield</span> <span class="n">ElementAndRestriction</span><span class="p">(</span>
<span class="n">element</span><span class="p">,</span> <span class="n">initial_restriction</span><span class="p">,</span> <span class="n">watermark_estimator_state</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="SplitRestrictionFn"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SplitRestrictionFn">[docs]</a><span class="k">class</span> <span class="nc">SplitRestrictionFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A transform that perform initial splitting of Splittable DoFn inputs.&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">do_fn</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_do_fn</span> <span class="o">=</span> <span class="n">do_fn</span>
<div class="viewcode-block" id="SplitRestrictionFn.start_bundle"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SplitRestrictionFn.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="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">_do_fn</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_invoker</span> <span class="o">=</span> <span class="n">DoFnInvoker</span><span class="o">.</span><span class="n">create_invoker</span><span class="p">(</span>
<span class="n">signature</span><span class="p">,</span>
<span class="n">output_processor</span><span class="o">=</span><span class="n">_NoneShallPassOutputHandler</span><span class="p">(),</span>
<span class="n">process_invocation</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span></div>
<div class="viewcode-block" id="SplitRestrictionFn.process"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SplitRestrictionFn.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_and_restriction</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">element</span> <span class="o">=</span> <span class="n">element_and_restriction</span><span class="o">.</span><span class="n">element</span>
<span class="n">restriction</span> <span class="o">=</span> <span class="n">element_and_restriction</span><span class="o">.</span><span class="n">restriction</span>
<span class="n">restriction_parts</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_invoker</span><span class="o">.</span><span class="n">invoke_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">for</span> <span class="n">part</span> <span class="ow">in</span> <span class="n">restriction_parts</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">ElementAndRestriction</span><span class="p">(</span>
<span class="n">element</span><span class="p">,</span> <span class="n">part</span><span class="p">,</span> <span class="n">element_and_restriction</span><span class="o">.</span><span class="n">watermark_estimator_state</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="ExplodeWindowsFn"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ExplodeWindowsFn">[docs]</a><span class="k">class</span> <span class="nc">ExplodeWindowsFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A transform that forces the runner to explode windows.</span>
<span class="sd"> This is done to make sure that Splittable DoFn proceses an element for each of</span>
<span class="sd"> the windows that element belongs to.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="ExplodeWindowsFn.process"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ExplodeWindowsFn.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">beam</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="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">yield</span> <span class="n">element</span></div></div>
<div class="viewcode-block" id="RandomUniqueKeyFn"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.RandomUniqueKeyFn">[docs]</a><span class="k">class</span> <span class="nc">RandomUniqueKeyFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A transform that assigns a unique key to each element.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="RandomUniqueKeyFn.process"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.RandomUniqueKeyFn.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">beam</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="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"># We ignore UUID collisions here since they are extremely rare.</span>
<span class="k">yield</span> <span class="p">(</span><span class="n">uuid</span><span class="o">.</span><span class="n">uuid4</span><span class="p">()</span><span class="o">.</span><span class="n">bytes</span><span class="p">,</span> <span class="n">element</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="ProcessKeyedElements"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessKeyedElements">[docs]</a><span class="k">class</span> <span class="nc">ProcessKeyedElements</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 primitive transform that performs SplittableDoFn magic.</span>
<span class="sd"> Input to this transform should be a PCollection of keyed ElementAndRestriction</span>
<span class="sd"> objects.</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">sdf</span><span class="p">,</span>
<span class="n">element_coder</span><span class="p">,</span>
<span class="n">restriction_coder</span><span class="p">,</span>
<span class="n">windowing_strategy</span><span class="p">,</span>
<span class="n">ptransform_args</span><span class="p">,</span>
<span class="n">ptransform_kwargs</span><span class="p">,</span>
<span class="n">ptransform_side_inputs</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span>
<span class="bp">self</span><span class="o">.</span><span class="n">element_coder</span> <span class="o">=</span> <span class="n">element_coder</span>
<span class="bp">self</span><span class="o">.</span><span class="n">restriction_coder</span> <span class="o">=</span> <span class="n">restriction_coder</span>
<span class="bp">self</span><span class="o">.</span><span class="n">windowing_strategy</span> <span class="o">=</span> <span class="n">windowing_strategy</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ptransform_args</span> <span class="o">=</span> <span class="n">ptransform_args</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ptransform_kwargs</span> <span class="o">=</span> <span class="n">ptransform_kwargs</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ptransform_side_inputs</span> <span class="o">=</span> <span class="n">ptransform_side_inputs</span>
<div class="viewcode-block" id="ProcessKeyedElements.expand"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessKeyedElements.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">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>
<div class="viewcode-block" id="ProcessKeyedElementsViaKeyedWorkItemsOverride"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessKeyedElementsViaKeyedWorkItemsOverride">[docs]</a><span class="k">class</span> <span class="nc">ProcessKeyedElementsViaKeyedWorkItemsOverride</span><span class="p">(</span><span class="n">PTransformOverride</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A transform override for ProcessElements transform.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="ProcessKeyedElementsViaKeyedWorkItemsOverride.matches"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessKeyedElementsViaKeyedWorkItemsOverride.matches">[docs]</a> <span class="k">def</span> <span class="nf">matches</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="n">ProcessKeyedElements</span><span class="p">)</span></div>
<div class="viewcode-block" id="ProcessKeyedElementsViaKeyedWorkItemsOverride.get_replacement_transform_for_applied_ptransform"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessKeyedElementsViaKeyedWorkItemsOverride.get_replacement_transform_for_applied_ptransform">[docs]</a> <span class="k">def</span> <span class="nf">get_replacement_transform_for_applied_ptransform</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">return</span> <span class="n">ProcessKeyedElementsViaKeyedWorkItems</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="ProcessKeyedElementsViaKeyedWorkItems"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessKeyedElementsViaKeyedWorkItems">[docs]</a><span class="k">class</span> <span class="nc">ProcessKeyedElementsViaKeyedWorkItems</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 processes Splittable DoFn input via KeyedWorkItems.&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">process_keyed_elements_transform</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span> <span class="o">=</span> <span class="n">process_keyed_elements_transform</span>
<div class="viewcode-block" id="ProcessKeyedElementsViaKeyedWorkItems.expand"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessKeyedElementsViaKeyedWorkItems.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">process_elements</span> <span class="o">=</span> <span class="n">ProcessElements</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span><span class="p">)</span>
<span class="n">process_elements</span><span class="o">.</span><span class="n">args</span> <span class="o">=</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span><span class="o">.</span><span class="n">ptransform_args</span><span class="p">)</span>
<span class="n">process_elements</span><span class="o">.</span><span class="n">kwargs</span> <span class="o">=</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span><span class="o">.</span><span class="n">ptransform_kwargs</span><span class="p">)</span>
<span class="n">process_elements</span><span class="o">.</span><span class="n">side_inputs</span> <span class="o">=</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span><span class="o">.</span><span class="n">ptransform_side_inputs</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">GroupByKey</span><span class="p">()</span> <span class="o">|</span> <span class="n">process_elements</span></div></div>
<div class="viewcode-block" id="ProcessElements"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessElements">[docs]</a><span class="k">class</span> <span class="nc">ProcessElements</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 primitive transform for processing keyed elements or KeyedWorkItems.</span>
<span class="sd"> Will be evaluated by</span>
<span class="sd"> `runners.direct.transform_evaluator._ProcessElementsEvaluator`.</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">process_keyed_elements_transform</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span> <span class="o">=</span> <span class="n">process_keyed_elements_transform</span>
<span class="bp">self</span><span class="o">.</span><span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span><span class="o">.</span><span class="n">sdf</span>
<div class="viewcode-block" id="ProcessElements.expand"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessElements.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">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="ProcessElements.new_process_fn"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessElements.new_process_fn">[docs]</a> <span class="k">def</span> <span class="nf">new_process_fn</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">sdf</span><span class="p">):</span>
<span class="k">return</span> <span class="n">ProcessFn</span><span class="p">(</span>
<span class="n">sdf</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span><span class="o">.</span><span class="n">ptransform_args</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_keyed_elements_transform</span><span class="o">.</span><span class="n">ptransform_kwargs</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="ProcessFn"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessFn">[docs]</a><span class="k">class</span> <span class="nc">ProcessFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A `DoFn` that executes machineary for invoking a Splittable `DoFn`.</span>
<span class="sd"> Input to the `ParDo` step that includes a `ProcessFn` will be a `PCollection`</span>
<span class="sd"> of `ElementAndRestriction` objects.</span>
<span class="sd"> This class is mainly responsible for following.</span>
<span class="sd"> (1) setup environment for properly invoking a Splittable `DoFn`.</span>
<span class="sd"> (2) invoke `process()` method of a Splittable `DoFn`.</span>
<span class="sd"> (3) after the `process()` invocation of the Splittable `DoFn`, determine if a</span>
<span class="sd"> re-invocation of the element is needed. If this is the case, set state and</span>
<span class="sd"> a timer for a re-invocation and hold output watermark till this</span>
<span class="sd"> re-invocation.</span>
<span class="sd"> (4) after the final invocation of a given element clear any previous state set</span>
<span class="sd"> for re-invoking the element and release the output watermark.</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">sdf</span><span class="p">,</span> <span class="n">args_for_invoker</span><span class="p">,</span> <span class="n">kwargs_for_invoker</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_element_tag</span> <span class="o">=</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">(</span><span class="s1">&#39;element&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_restriction_tag</span> <span class="o">=</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">(</span><span class="s1">&#39;restriction&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_watermark_state_tag</span> <span class="o">=</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">(</span>
<span class="s1">&#39;watermark_estimator_state&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">watermark_hold_tag</span> <span class="o">=</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">(</span><span class="s1">&#39;watermark_hold&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_element_invoker</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_output_processor</span> <span class="o">=</span> <span class="n">_OutputHandler</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">sdf_invoker</span> <span class="o">=</span> <span class="n">DoFnInvoker</span><span class="o">.</span><span class="n">create_invoker</span><span class="p">(</span>
<span class="n">DoFnSignature</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">sdf</span><span class="p">),</span>
<span class="n">context</span><span class="o">=</span><span class="n">DoFnContext</span><span class="p">(</span><span class="s1">&#39;unused_context&#39;</span><span class="p">),</span>
<span class="n">output_processor</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_output_processor</span><span class="p">,</span>
<span class="n">input_args</span><span class="o">=</span><span class="n">args_for_invoker</span><span class="p">,</span>
<span class="n">input_kwargs</span><span class="o">=</span><span class="n">kwargs_for_invoker</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_step_context</span> <span class="o">=</span> <span class="kc">None</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">step_context</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">_step_context</span>
<span class="nd">@step_context</span><span class="o">.</span><span class="n">setter</span>
<span class="k">def</span> <span class="nf">step_context</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">step_context</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">step_context</span><span class="p">,</span> <span class="n">DirectStepContext</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_step_context</span> <span class="o">=</span> <span class="n">step_context</span>
<div class="viewcode-block" id="ProcessFn.set_process_element_invoker"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessFn.set_process_element_invoker">[docs]</a> <span class="k">def</span> <span class="nf">set_process_element_invoker</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">process_element_invoker</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">process_element_invoker</span><span class="p">,</span> <span class="n">SDFProcessElementInvoker</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_process_element_invoker</span> <span class="o">=</span> <span class="n">process_element_invoker</span></div>
<div class="viewcode-block" id="ProcessFn.process"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.ProcessFn.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">beam</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">beam</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="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">isinstance</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">KeyedWorkItem</span><span class="p">):</span>
<span class="c1"># Must be a timer firing.</span>
<span class="n">key</span> <span class="o">=</span> <span class="n">element</span><span class="o">.</span><span class="n">encoded_key</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">key</span><span class="p">,</span> <span class="n">values</span> <span class="o">=</span> <span class="n">element</span>
<span class="n">values</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">values</span><span class="p">)</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">values</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span>
<span class="c1"># Value here will either be a WindowedValue or an ElementAndRestriction</span>
<span class="c1"># object.</span>
<span class="c1"># TODO: handle key collisions here.</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">values</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">,</span> <span class="s1">&#39;Internal error. Processing of splittable &#39;</span> \
<span class="s1">&#39;DoFn cannot continue since elements did not &#39;</span> \
<span class="s1">&#39;have unique keys.&#39;</span>
<span class="n">value</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="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">values</span><span class="p">)</span> <span class="o">!=</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;&#39;</span><span class="p">)</span>
<span class="n">state</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_step_context</span><span class="o">.</span><span class="n">get_keyed_state</span><span class="p">(</span><span class="n">key</span><span class="p">)</span>
<span class="n">element_state</span> <span class="o">=</span> <span class="n">state</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_element_tag</span><span class="p">)</span>
<span class="c1"># Initially element_state is an empty list.</span>
<span class="n">is_seed_call</span> <span class="o">=</span> <span class="ow">not</span> <span class="n">element_state</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_seed_call</span><span class="p">:</span>
<span class="n">element</span> <span class="o">=</span> <span class="n">state</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_element_tag</span><span class="p">)</span>
<span class="n">restriction</span> <span class="o">=</span> <span class="n">state</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_restriction_tag</span><span class="p">)</span>
<span class="n">watermark_estimator_state</span> <span class="o">=</span> <span class="n">state</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_watermark_state_tag</span><span class="p">)</span>
<span class="n">windowed_element</span> <span class="o">=</span> <span class="n">WindowedValue</span><span class="p">(</span><span class="n">element</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>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># After values iterator is expanded above we should have gotten a list</span>
<span class="c1"># with a single ElementAndRestriction object.</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">ElementAndRestriction</span><span class="p">)</span>
<span class="n">element_and_restriction</span> <span class="o">=</span> <span class="n">value</span>
<span class="n">element</span> <span class="o">=</span> <span class="n">element_and_restriction</span><span class="o">.</span><span class="n">element</span>
<span class="n">restriction</span> <span class="o">=</span> <span class="n">element_and_restriction</span><span class="o">.</span><span class="n">restriction</span>
<span class="n">watermark_estimator_state</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">element_and_restriction</span><span class="o">.</span><span class="n">watermark_estimator_state</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">WindowedValue</span><span class="p">):</span>
<span class="n">windowed_element</span> <span class="o">=</span> <span class="n">WindowedValue</span><span class="p">(</span>
<span class="n">element</span><span class="p">,</span> <span class="n">value</span><span class="o">.</span><span class="n">timestamp</span><span class="p">,</span> <span class="n">value</span><span class="o">.</span><span class="n">windows</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">windowed_element</span> <span class="o">=</span> <span class="n">WindowedValue</span><span class="p">(</span><span class="n">element</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>
<span class="k">assert</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_element_invoker</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_process_element_invoker</span><span class="p">,</span> <span class="n">SDFProcessElementInvoker</span><span class="p">)</span>
<span class="n">output_values</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_process_element_invoker</span><span class="o">.</span><span class="n">invoke_process_element</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">sdf_invoker</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_output_processor</span><span class="p">,</span>
<span class="n">windowed_element</span><span class="p">,</span>
<span class="n">restriction</span><span class="p">,</span>
<span class="n">watermark_estimator_state</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">sdf_result</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">for</span> <span class="n">output</span> <span class="ow">in</span> <span class="n">output_values</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">output</span><span class="p">,</span> <span class="n">SDFProcessElementInvoker</span><span class="o">.</span><span class="n">Result</span><span class="p">):</span>
<span class="c1"># SDFProcessElementInvoker.Result should be the last item yielded.</span>
<span class="n">sdf_result</span> <span class="o">=</span> <span class="n">output</span>
<span class="k">break</span>
<span class="k">yield</span> <span class="n">output</span>
<span class="k">assert</span> <span class="n">sdf_result</span><span class="p">,</span> <span class="p">(</span><span class="s1">&#39;SDFProcessElementInvoker must return a &#39;</span>
<span class="s1">&#39;SDFProcessElementInvoker.Result object as the last &#39;</span>
<span class="s1">&#39;value of a SDF invoke_process_element() invocation.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">sdf_result</span><span class="o">.</span><span class="n">residual_restriction</span><span class="p">:</span>
<span class="c1"># All work for current residual and restriction pair is complete.</span>
<span class="n">state</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_element_tag</span><span class="p">)</span>
<span class="n">state</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_restriction_tag</span><span class="p">)</span>
<span class="n">state</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_watermark_state_tag</span><span class="p">)</span>
<span class="c1"># Releasing output watermark by setting it to positive infinity.</span>
<span class="n">state</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">watermark_hold_tag</span><span class="p">,</span> <span class="n">WatermarkManager</span><span class="o">.</span><span class="n">WATERMARK_POS_INF</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">state</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_element_tag</span><span class="p">,</span> <span class="n">element</span><span class="p">)</span>
<span class="n">state</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_restriction_tag</span><span class="p">,</span> <span class="n">sdf_result</span><span class="o">.</span><span class="n">residual_restriction</span><span class="p">)</span>
<span class="n">state</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_watermark_state_tag</span><span class="p">,</span> <span class="n">watermark_estimator_state</span><span class="p">)</span>
<span class="c1"># Holding output watermark by setting it to negative infinity.</span>
<span class="n">state</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">watermark_hold_tag</span><span class="p">,</span> <span class="n">WatermarkManager</span><span class="o">.</span><span class="n">WATERMARK_NEG_INF</span><span class="p">)</span>
<span class="c1"># Setting a timer to be reinvoked to continue processing the element.</span>
<span class="c1"># Currently Python SDK only supports setting timers based on watermark. So</span>
<span class="c1"># forcing a reinvocation by setting a timer for watermark negative</span>
<span class="c1"># infinity.</span>
<span class="c1"># TODO(chamikara): update this by setting a timer for the proper</span>
<span class="c1"># processing time when Python SDK supports that.</span>
<span class="n">state</span><span class="o">.</span><span class="n">set_timer</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">,</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">WATERMARK</span><span class="p">,</span> <span class="n">WatermarkManager</span><span class="o">.</span><span class="n">WATERMARK_NEG_INF</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="SDFProcessElementInvoker"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SDFProcessElementInvoker">[docs]</a><span class="k">class</span> <span class="nc">SDFProcessElementInvoker</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 utility that invokes SDF `process()` method and requests checkpoints.</span>
<span class="sd"> This class is responsible for invoking the `process()` method of a Splittable</span>
<span class="sd"> `DoFn` and making sure that invocation terminated properly. Based on the input</span>
<span class="sd"> configuration, this class may decide to request a checkpoint for a `process()`</span>
<span class="sd"> execution so that runner can process current output and resume the invocation</span>
<span class="sd"> at a later time.</span>
<span class="sd"> More specifically, when initializing a `SDFProcessElementInvoker`, caller may</span>
<span class="sd"> specify the number of output elements or processing time after which a</span>
<span class="sd"> checkpoint should be requested. This class is responsible for properly</span>
<span class="sd"> requesting a checkpoint based on either of these criteria.</span>
<span class="sd"> When the `process()` call of Splittable `DoFn` ends, this class performs</span>
<span class="sd"> validations to make sure that processing ended gracefully and returns a</span>
<span class="sd"> `SDFProcessElementInvoker.Result` that contains information which can be used</span>
<span class="sd"> by the caller to perform another `process()` invocation for the residual.</span>
<span class="sd"> A `process()` invocation may decide to give up processing voluntarily by</span>
<span class="sd"> returning a `ProcessContinuation` object (see documentation of</span>
<span class="sd"> `ProcessContinuation` for more details). So if a &#39;ProcessContinuation&#39; is</span>
<span class="sd"> produced this class ends the execution and performs steps to finalize the</span>
<span class="sd"> current invocation.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="SDFProcessElementInvoker.Result"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SDFProcessElementInvoker.Result">[docs]</a> <span class="k">class</span> <span class="nc">Result</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">residual_restriction</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">process_continuation</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">future_output_watermark</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returned as a result of a `invoke_process_element()` invocation.</span>
<span class="sd"> Args:</span>
<span class="sd"> residual_restriction: a restriction for the unprocessed part of the</span>
<span class="sd"> element.</span>
<span class="sd"> process_continuation: a `ProcessContinuation` if one was returned as the</span>
<span class="sd"> last element of the SDF `process()` invocation.</span>
<span class="sd"> future_output_watermark: output watermark of the results that will be</span>
<span class="sd"> produced when invoking the Splittable `DoFn`</span>
<span class="sd"> for the current element with</span>
<span class="sd"> `residual_restriction`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">residual_restriction</span> <span class="o">=</span> <span class="n">residual_restriction</span>
<span class="bp">self</span><span class="o">.</span><span class="n">process_continuation</span> <span class="o">=</span> <span class="n">process_continuation</span>
<span class="bp">self</span><span class="o">.</span><span class="n">future_output_watermark</span> <span class="o">=</span> <span class="n">future_output_watermark</span></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">max_num_outputs</span><span class="p">,</span> <span class="n">max_duration</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_max_num_outputs</span> <span class="o">=</span> <span class="n">max_num_outputs</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_max_duration</span> <span class="o">=</span> <span class="n">max_duration</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_checkpoint_lock</span> <span class="o">=</span> <span class="n">Lock</span><span class="p">()</span>
<div class="viewcode-block" id="SDFProcessElementInvoker.test_method"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SDFProcessElementInvoker.test_method">[docs]</a> <span class="k">def</span> <span class="nf">test_method</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span></div>
<div class="viewcode-block" id="SDFProcessElementInvoker.invoke_process_element"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.sdf_direct_runner.html#apache_beam.runners.direct.sdf_direct_runner.SDFProcessElementInvoker.invoke_process_element">[docs]</a> <span class="k">def</span> <span class="nf">invoke_process_element</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">sdf_invoker</span><span class="p">,</span>
<span class="n">output_processor</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="n">watermark_estimator_state</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;Invokes `process()` method of a Splittable `DoFn` for a given element.</span>
<span class="sd"> Args:</span>
<span class="sd"> sdf_invoker: a `DoFnInvoker` for the Splittable `DoFn`.</span>
<span class="sd"> element: the element to process</span>
<span class="sd"> Returns:</span>
<span class="sd"> a `SDFProcessElementInvoker.Result` object.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">sdf_invoker</span><span class="p">,</span> <span class="n">DoFnInvoker</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">CheckpointState</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="bp">self</span><span class="o">.</span><span class="n">checkpointed</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">residual_restriction</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">checkpoint_state</span> <span class="o">=</span> <span class="n">CheckpointState</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">initiate_checkpoint</span><span class="p">():</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_checkpoint_lock</span><span class="p">:</span>
<span class="k">if</span> <span class="n">checkpoint_state</span><span class="o">.</span><span class="n">checkpointed</span><span class="p">:</span>
<span class="k">return</span>
<span class="n">checkpoint_state</span><span class="o">.</span><span class="n">checkpointed</span> <span class="o">=</span> <span class="nb">object</span><span class="p">()</span>
<span class="n">split</span> <span class="o">=</span> <span class="n">sdf_invoker</span><span class="o">.</span><span class="n">try_split</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span>
<span class="k">if</span> <span class="n">split</span><span class="p">:</span>
<span class="n">_</span><span class="p">,</span> <span class="n">checkpoint_state</span><span class="o">.</span><span class="n">residual_restriction</span> <span class="o">=</span> <span class="n">split</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Clear the checkpoint if the split didn&#39;t happen. This counters</span>
<span class="c1"># a very unlikely race condition that the Timer attempted to initiate</span>
<span class="c1"># a checkpoint before invoke_process set the current element allowing</span>
<span class="c1"># for another attempt to checkpoint.</span>
<span class="n">checkpoint_state</span><span class="o">.</span><span class="n">checkpointed</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">output_processor</span><span class="o">.</span><span class="n">reset</span><span class="p">()</span>
<span class="n">Timer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_max_duration</span><span class="p">,</span> <span class="n">initiate_checkpoint</span><span class="p">)</span><span class="o">.</span><span class="n">start</span><span class="p">()</span>
<span class="n">sdf_invoker</span><span class="o">.</span><span class="n">invoke_process</span><span class="p">(</span>
<span class="n">element</span><span class="p">,</span>
<span class="n">additional_args</span><span class="o">=</span><span class="n">args</span><span class="p">,</span>
<span class="n">restriction</span><span class="o">=</span><span class="n">restriction</span><span class="p">,</span>
<span class="n">watermark_estimator_state</span><span class="o">=</span><span class="n">watermark_estimator_state</span><span class="p">)</span>
<span class="k">assert</span> <span class="n">output_processor</span><span class="o">.</span><span class="n">output_iter</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="n">output_count</span> <span class="o">=</span> <span class="mi">0</span>
<span class="c1"># We have to expand and re-yield here to support ending execution for a</span>
<span class="c1"># given number of output elements as well as to capture the</span>
<span class="c1"># ProcessContinuation of one was returned.</span>
<span class="n">process_continuation</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">for</span> <span class="n">output</span> <span class="ow">in</span> <span class="n">output_processor</span><span class="o">.</span><span class="n">output_iter</span><span class="p">:</span>
<span class="c1"># A ProcessContinuation, if returned, should be the last element.</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">process_continuation</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">output</span><span class="p">,</span> <span class="n">ProcessContinuation</span><span class="p">):</span>
<span class="c1"># Taking a checkpoint so that we can determine primary and residual</span>
<span class="c1"># restrictions.</span>
<span class="n">initiate_checkpoint</span><span class="p">()</span>
<span class="c1"># A ProcessContinuation should always be the last element produced by</span>
<span class="c1"># the output iterator.</span>
<span class="c1"># TODO: support continuing after the specified amount of delay.</span>
<span class="c1"># Continuing here instead of breaking to enforce that this is the last</span>
<span class="c1"># element.</span>
<span class="n">process_continuation</span> <span class="o">=</span> <span class="n">output</span>
<span class="k">continue</span>
<span class="k">yield</span> <span class="n">output</span>
<span class="n">output_count</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_num_outputs</span> <span class="ow">and</span> <span class="n">output_count</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_num_outputs</span><span class="p">:</span>
<span class="n">initiate_checkpoint</span><span class="p">()</span>
<span class="n">result</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">SDFProcessElementInvoker</span><span class="o">.</span><span class="n">Result</span><span class="p">(</span>
<span class="n">residual_restriction</span><span class="o">=</span><span class="n">checkpoint_state</span><span class="o">.</span><span class="n">residual_restriction</span><span class="p">)</span>
<span class="k">if</span> <span class="n">checkpoint_state</span><span class="o">.</span><span class="n">residual_restriction</span> <span class="k">else</span>
<span class="n">SDFProcessElementInvoker</span><span class="o">.</span><span class="n">Result</span><span class="p">())</span>
<span class="k">yield</span> <span class="n">result</span></div></div>
<span class="k">class</span> <span class="nc">_OutputHandler</span><span class="p">(</span><span class="n">OutputHandler</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="bp">self</span><span class="o">.</span><span class="n">output_iter</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">process_outputs</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">windowed_input_element</span><span class="p">,</span> <span class="n">output_iter</span><span class="p">,</span> <span class="n">watermark_estimator</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="c1"># type: (WindowedValue, Iterable[Any], Optional[WatermarkEstimator]) -&gt; None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">output_iter</span> <span class="o">=</span> <span class="n">output_iter</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="bp">self</span><span class="o">.</span><span class="n">output_iter</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">class</span> <span class="nc">_NoneShallPassOutputHandler</span><span class="p">(</span><span class="n">OutputHandler</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">process_outputs</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">windowed_input_element</span><span class="p">,</span> <span class="n">output_iter</span><span class="p">,</span> <span class="n">watermark_estimator</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="c1"># type: (WindowedValue, Iterable[Any], Optional[WatermarkEstimator]) -&gt; None</span>
<span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">()</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>