blob: 6615e1740ff793f9749ea177dea8e979abba65ea [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.trigger &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.trigger</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.trigger</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;Support for Apache Beam triggers.</span>
<span class="sd">Triggers control when in processing time windows get emitted.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span> <span class="nn">collections</span>
<span class="kn">import</span> <span class="nn">copy</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">import</span> <span class="nn">numbers</span>
<span class="kn">from</span> <span class="nn">abc</span> <span class="kn">import</span> <span class="n">ABCMeta</span>
<span class="kn">from</span> <span class="nn">abc</span> <span class="kn">import</span> <span class="n">abstractmethod</span>
<span class="kn">from</span> <span class="nn">collections</span> <span class="kn">import</span> <span class="n">abc</span> <span class="k">as</span> <span class="n">collections_abc</span> <span class="c1"># ambiguty with direct abc</span>
<span class="kn">from</span> <span class="nn">enum</span> <span class="kn">import</span> <span class="n">Flag</span>
<span class="kn">from</span> <span class="nn">enum</span> <span class="kn">import</span> <span class="n">auto</span>
<span class="kn">from</span> <span class="nn">itertools</span> <span class="kn">import</span> <span class="n">zip_longest</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders</span> <span class="kn">import</span> <span class="n">coder_impl</span>
<span class="kn">from</span> <span class="nn">apache_beam.coders</span> <span class="kn">import</span> <span class="n">observable</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">combiners</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">core</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.timeutil</span> <span class="kn">import</span> <span class="n">TimeDomain</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">GlobalWindow</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">TimestampCombiner</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.utils</span> <span class="kn">import</span> <span class="n">windowed_value</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.timestamp</span> <span class="kn">import</span> <span class="n">MAX_TIMESTAMP</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.timestamp</span> <span class="kn">import</span> <span class="n">MIN_TIMESTAMP</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.timestamp</span> <span class="kn">import</span> <span class="n">TIME_GRANULARITY</span>
<span class="c1"># AfterCount is experimental. No backwards compatibility guarantees.</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span>
<span class="s1">&#39;AccumulationMode&#39;</span><span class="p">,</span>
<span class="s1">&#39;TriggerFn&#39;</span><span class="p">,</span>
<span class="s1">&#39;DefaultTrigger&#39;</span><span class="p">,</span>
<span class="s1">&#39;AfterWatermark&#39;</span><span class="p">,</span>
<span class="s1">&#39;AfterProcessingTime&#39;</span><span class="p">,</span>
<span class="s1">&#39;AfterCount&#39;</span><span class="p">,</span>
<span class="s1">&#39;Repeatedly&#39;</span><span class="p">,</span>
<span class="s1">&#39;AfterAny&#39;</span><span class="p">,</span>
<span class="s1">&#39;AfterAll&#39;</span><span class="p">,</span>
<span class="s1">&#39;AfterEach&#39;</span><span class="p">,</span>
<span class="s1">&#39;OrFinally&#39;</span><span class="p">,</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>
<div class="viewcode-block" id="AccumulationMode"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AccumulationMode">[docs]</a><span class="k">class</span> <span class="nc">AccumulationMode</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Controls what to do with data when a trigger fires multiple times.&quot;&quot;&quot;</span>
<span class="n">DISCARDING</span> <span class="o">=</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">AccumulationMode</span><span class="o">.</span><span class="n">DISCARDING</span>
<span class="n">ACCUMULATING</span> <span class="o">=</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">AccumulationMode</span><span class="o">.</span><span class="n">ACCUMULATING</span></div>
<span class="c1"># TODO(robertwb): Provide retractions of previous outputs.</span>
<span class="c1"># RETRACTING = 3</span>
<span class="k">class</span> <span class="nc">_StateTag</span><span class="p">(</span><span class="n">metaclass</span><span class="o">=</span><span class="n">ABCMeta</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;An identifier used to store and retrieve typed, combinable state.</span>
<span class="sd"> The given tag must be unique for this step.&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">tag</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">tag</span> <span class="o">=</span> <span class="n">tag</span>
<span class="k">class</span> <span class="nc">_ReadModifyWriteStateTag</span><span class="p">(</span><span class="n">_StateTag</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;StateTag pointing to an element.&quot;&quot;&quot;</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;ValueStateTag(</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">tag</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">with_prefix</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">prefix</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">(</span><span class="n">prefix</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_SetStateTag</span><span class="p">(</span><span class="n">_StateTag</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;StateTag pointing to an element.&quot;&quot;&quot;</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;SetStateTag(</span><span class="si">{tag}</span><span class="s1">)&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">tag</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">with_prefix</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">prefix</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_SetStateTag</span><span class="p">(</span><span class="n">prefix</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_CombiningValueStateTag</span><span class="p">(</span><span class="n">_StateTag</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;StateTag pointing to an element, accumulated with a combiner.</span>
<span class="sd"> The given tag must be unique for this step. The given CombineFn will be</span>
<span class="sd"> applied (possibly incrementally and eagerly) when adding elements.&quot;&quot;&quot;</span>
<span class="c1"># TODO(robertwb): Also store the coder (perhaps extracted from the combine_fn)</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">tag</span><span class="p">,</span> <span class="n">combine_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="n">tag</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">combine_fn</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;combine_fn must be specified.&#39;</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">combine_fn</span><span class="p">,</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="n">combine_fn</span> <span class="o">=</span> <span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="o">.</span><span class="n">from_callable</span><span class="p">(</span><span class="n">combine_fn</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span> <span class="o">=</span> <span class="n">combine_fn</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;CombiningValueStateTag(</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">tag</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="k">def</span> <span class="nf">with_prefix</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">prefix</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_CombiningValueStateTag</span><span class="p">(</span><span class="n">prefix</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">without_extraction</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">class</span> <span class="nc">NoExtractionCombineFn</span><span class="p">(</span><span class="n">core</span><span class="o">.</span><span class="n">CombineFn</span><span class="p">):</span>
<span class="n">setup</span> <span class="o">=</span> <span class="bp">self</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="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">create_accumulator</span>
<span class="n">add_input</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">add_input</span>
<span class="n">merge_accumulators</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">merge_accumulators</span>
<span class="n">compact</span> <span class="o">=</span> <span class="bp">self</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="nb">staticmethod</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">)</span>
<span class="n">teardown</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">teardown</span>
<span class="k">return</span> <span class="n">_CombiningValueStateTag</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">,</span> <span class="n">NoExtractionCombineFn</span><span class="p">())</span>
<span class="k">class</span> <span class="nc">_ListStateTag</span><span class="p">(</span><span class="n">_StateTag</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;StateTag pointing to a list of elements.&quot;&quot;&quot;</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;ListStateTag(</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">tag</span>
<span class="k">def</span> <span class="nf">with_prefix</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">prefix</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_ListStateTag</span><span class="p">(</span><span class="n">prefix</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_WatermarkHoldStateTag</span><span class="p">(</span><span class="n">_StateTag</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">tag</span><span class="p">,</span> <span class="n">timestamp_combiner_impl</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">tag</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner_impl</span> <span class="o">=</span> <span class="n">timestamp_combiner_impl</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;WatermarkHoldStateTag(</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">tag</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner_impl</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">with_prefix</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">prefix</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_WatermarkHoldStateTag</span><span class="p">(</span>
<span class="n">prefix</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">tag</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner_impl</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">DataLossReason</span><span class="p">(</span><span class="n">Flag</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Enum defining potential reasons that a trigger may cause data loss.</span>
<span class="sd"> These flags should only cover when the trigger is the cause, though windowing</span>
<span class="sd"> can be taken into account. For instance, AfterWatermark may not flag itself</span>
<span class="sd"> as finishing if the windowing doesn&#39;t allow lateness.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Trigger will never be the source of data loss.</span>
<span class="n">NO_POTENTIAL_LOSS</span> <span class="o">=</span> <span class="mi">0</span>
<span class="c1"># Trigger may finish. In this case, data that comes in after the trigger may</span>
<span class="c1"># be lost. Example: AfterCount(1) will stop firing after the first element.</span>
<span class="n">MAY_FINISH</span> <span class="o">=</span> <span class="n">auto</span><span class="p">()</span>
<span class="c1"># Deprecated: Beam will emit buffered data at GC time. Any other behavior</span>
<span class="c1"># should be treated as a bug with the runner used.</span>
<span class="n">CONDITION_NOT_GUARANTEED</span> <span class="o">=</span> <span class="n">auto</span><span class="p">()</span>
<span class="c1"># Convenience functions for checking if a flag is included. Each is equivalent</span>
<span class="c1"># to `reason &amp; flag == flag`</span>
<span class="k">def</span> <span class="nf">_IncludesMayFinish</span><span class="p">(</span><span class="n">reason</span><span class="p">):</span>
<span class="c1"># type: (DataLossReason) -&gt; bool</span>
<span class="k">return</span> <span class="n">reason</span> <span class="o">&amp;</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">MAY_FINISH</span> <span class="o">==</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">MAY_FINISH</span>
<span class="c1"># pylint: disable=unused-argument</span>
<span class="c1"># TODO(robertwb): Provisional API, Java likely to change as well.</span>
<div class="viewcode-block" id="TriggerFn"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn">[docs]</a><span class="k">class</span> <span class="nc">TriggerFn</span><span class="p">(</span><span class="n">metaclass</span><span class="o">=</span><span class="n">ABCMeta</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A TriggerFn determines when window (panes) are emitted.</span>
<span class="sd"> See https://beam.apache.org/documentation/programming-guide/#triggers</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="TriggerFn.on_element"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.on_element">[docs]</a> <span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">on_element</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="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Called when a new element arrives in a window.</span>
<span class="sd"> Args:</span>
<span class="sd"> element: the element being added</span>
<span class="sd"> window: the window to which the element is being added</span>
<span class="sd"> context: a context (e.g. a TriggerContext instance) for managing state</span>
<span class="sd"> and setting timers</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="TriggerFn.on_merge"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.on_merge">[docs]</a> <span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Called when multiple windows are merged.</span>
<span class="sd"> Args:</span>
<span class="sd"> to_be_merged: the set of windows to be merged</span>
<span class="sd"> merge_result: the window into which the windows are being merged</span>
<span class="sd"> context: a context (e.g. a TriggerContext instance) for managing state</span>
<span class="sd"> and setting timers</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="TriggerFn.should_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.should_fire">[docs]</a> <span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Whether this trigger should cause the window to fire.</span>
<span class="sd"> Args:</span>
<span class="sd"> time_domain: WATERMARK for event-time timers and REAL_TIME for</span>
<span class="sd"> processing-time timers.</span>
<span class="sd"> timestamp: for time_domain WATERMARK, it represents the</span>
<span class="sd"> watermark: (a lower bound on) the watermark of the system</span>
<span class="sd"> and for time_domain REAL_TIME, it represents the</span>
<span class="sd"> trigger: timestamp of the processing-time timer.</span>
<span class="sd"> window: the window whose trigger is being considered</span>
<span class="sd"> context: a context (e.g. a TriggerContext instance) for managing state</span>
<span class="sd"> and setting timers</span>
<span class="sd"> Returns:</span>
<span class="sd"> whether this trigger should cause a firing</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="TriggerFn.has_ontime_pane"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.has_ontime_pane">[docs]</a> <span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">has_ontime_pane</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Whether this trigger creates an empty pane even if there are no elements.</span>
<span class="sd"> Returns:</span>
<span class="sd"> True if this trigger guarantees that there will always be an ON_TIME pane</span>
<span class="sd"> even if there are no elements in that pane.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="TriggerFn.on_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.on_fire">[docs]</a> <span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Called when a trigger actually fires.</span>
<span class="sd"> Args:</span>
<span class="sd"> watermark: (a lower bound on) the watermark of the system</span>
<span class="sd"> window: the window whose trigger is being fired</span>
<span class="sd"> context: a context (e.g. a TriggerContext instance) for managing state</span>
<span class="sd"> and setting timers</span>
<span class="sd"> Returns:</span>
<span class="sd"> whether this trigger is finished</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="TriggerFn.reset"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.reset">[docs]</a> <span class="nd">@abstractmethod</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="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Clear any state and timers used by this TriggerFn.&quot;&quot;&quot;</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="TriggerFn.may_lose_data"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.may_lose_data">[docs]</a> <span class="k">def</span> <span class="nf">may_lose_data</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_windowing</span><span class="p">):</span>
<span class="c1"># type: (core.Windowing) -&gt; DataLossReason</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns whether or not this trigger could cause data loss.</span>
<span class="sd"> A trigger can cause data loss in the following scenarios:</span>
<span class="sd"> * The trigger has a chance to finish. For instance, AfterWatermark()</span>
<span class="sd"> without a late trigger would cause all late data to be lost. This</span>
<span class="sd"> scenario is only accounted for if the windowing strategy allows</span>
<span class="sd"> late data. Otherwise, the trigger is not responsible for the data</span>
<span class="sd"> loss.</span>
<span class="sd"> Note that this only returns the potential for loss. It does not mean that</span>
<span class="sd"> there will be data loss. It also only accounts for loss related to the</span>
<span class="sd"> trigger, not other potential causes.</span>
<span class="sd"> Args:</span>
<span class="sd"> windowing: The Windowing that this trigger belongs to. It does not need</span>
<span class="sd"> to be the top-level trigger.</span>
<span class="sd"> Returns:</span>
<span class="sd"> The DataLossReason. If there is no potential loss,</span>
<span class="sd"> DataLossReason.NO_POTENTIAL_LOSS is returned. Otherwise, all the</span>
<span class="sd"> potential reasons are returned as a single value.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># For backwards compatibility&#39;s sake, we&#39;re assuming the trigger is safe.</span>
<span class="k">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span></div>
<span class="c1"># pylint: enable=unused-argument</span>
<div class="viewcode-block" id="TriggerFn.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.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="k">return</span> <span class="p">{</span>
<span class="s1">&#39;after_all&#39;</span><span class="p">:</span> <span class="n">AfterAll</span><span class="p">,</span>
<span class="s1">&#39;after_any&#39;</span><span class="p">:</span> <span class="n">AfterAny</span><span class="p">,</span>
<span class="s1">&#39;after_each&#39;</span><span class="p">:</span> <span class="n">AfterEach</span><span class="p">,</span>
<span class="s1">&#39;after_end_of_window&#39;</span><span class="p">:</span> <span class="n">AfterWatermark</span><span class="p">,</span>
<span class="s1">&#39;after_processing_time&#39;</span><span class="p">:</span> <span class="n">AfterProcessingTime</span><span class="p">,</span>
<span class="c1"># after_processing_time, after_synchronized_processing_time</span>
<span class="s1">&#39;always&#39;</span><span class="p">:</span> <span class="n">Always</span><span class="p">,</span>
<span class="s1">&#39;default&#39;</span><span class="p">:</span> <span class="n">DefaultTrigger</span><span class="p">,</span>
<span class="s1">&#39;element_count&#39;</span><span class="p">:</span> <span class="n">AfterCount</span><span class="p">,</span>
<span class="s1">&#39;never&#39;</span><span class="p">:</span> <span class="n">_Never</span><span class="p">,</span>
<span class="s1">&#39;or_finally&#39;</span><span class="p">:</span> <span class="n">OrFinally</span><span class="p">,</span>
<span class="s1">&#39;repeat&#39;</span><span class="p">:</span> <span class="n">Repeatedly</span><span class="p">,</span>
<span class="p">}[</span><span class="n">proto</span><span class="o">.</span><span class="n">WhichOneof</span><span class="p">(</span><span class="s1">&#39;trigger&#39;</span><span class="p">)]</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></div>
<div class="viewcode-block" id="TriggerFn.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.TriggerFn.to_runner_api">[docs]</a> <span class="nd">@abstractmethod</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">pass</span></div></div>
<div class="viewcode-block" id="DefaultTrigger"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger">[docs]</a><span class="k">class</span> <span class="nc">DefaultTrigger</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Semantically Repeatedly(AfterWatermark()), but more optimized.&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="k">pass</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;DefaultTrigger()&#39;</span>
<div class="viewcode-block" id="DefaultTrigger.on_element"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.on_element">[docs]</a> <span class="k">def</span> <span class="nf">on_element</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="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">context</span><span class="o">.</span><span class="n">set_timer</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">window</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">window</span><span class="o">.</span><span class="n">end</span><span class="p">)</span></div>
<div class="viewcode-block" id="DefaultTrigger.on_merge"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.on_merge">[docs]</a> <span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">for</span> <span class="n">window</span> <span class="ow">in</span> <span class="n">to_be_merged</span><span class="p">:</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_timer</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">window</span><span class="p">),</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">WATERMARK</span><span class="p">)</span></div>
<div class="viewcode-block" id="DefaultTrigger.should_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.should_fire">[docs]</a> <span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">if</span> <span class="n">watermark</span> <span class="o">&gt;=</span> <span class="n">window</span><span class="o">.</span><span class="n">end</span><span class="p">:</span>
<span class="c1"># Explicitly clear the timer so that late elements are not emitted again</span>
<span class="c1"># when the timer is fired.</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_timer</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">window</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="k">return</span> <span class="n">watermark</span> <span class="o">&gt;=</span> <span class="n">window</span><span class="o">.</span><span class="n">end</span></div>
<div class="viewcode-block" id="DefaultTrigger.on_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.on_fire">[docs]</a> <span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span></div>
<div class="viewcode-block" id="DefaultTrigger.reset"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.reset">[docs]</a> <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="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_timer</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">window</span><span class="p">),</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">WATERMARK</span><span class="p">)</span></div>
<div class="viewcode-block" id="DefaultTrigger.may_lose_data"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.may_lose_data">[docs]</a> <span class="k">def</span> <span class="nf">may_lose_data</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_windowing</span><span class="p">):</span>
<span class="k">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span></div>
<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">return</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">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="nb">type</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span>
<div class="viewcode-block" id="DefaultTrigger.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.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="k">return</span> <span class="n">DefaultTrigger</span><span class="p">()</span></div>
<div class="viewcode-block" id="DefaultTrigger.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.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">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">Trigger</span><span class="p">(</span>
<span class="n">default</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">Default</span><span class="p">())</span></div>
<div class="viewcode-block" id="DefaultTrigger.has_ontime_pane"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.DefaultTrigger.has_ontime_pane">[docs]</a> <span class="k">def</span> <span class="nf">has_ontime_pane</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>
<div class="viewcode-block" id="AfterProcessingTime"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime">[docs]</a><span class="k">class</span> <span class="nc">AfterProcessingTime</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Fire exactly once after a specified delay from processing time.</span>
<span class="sd"> AfterProcessingTime is experimental. No backwards compatibility guarantees.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">STATE_TAG</span> <span class="o">=</span> <span class="n">_SetStateTag</span><span class="p">(</span><span class="s1">&#39;has_timer&#39;</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">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;Initialize a processing time trigger with a delay in seconds.&quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">delay</span> <span class="o">=</span> <span class="n">delay</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;AfterProcessingTime(delay=</span><span class="si">%d</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">delay</span>
<div class="viewcode-block" id="AfterProcessingTime.on_element"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.on_element">[docs]</a> <span class="k">def</span> <span class="nf">on_element</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="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">context</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">STATE_TAG</span><span class="p">):</span>
<span class="n">context</span><span class="o">.</span><span class="n">set_timer</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">REAL_TIME</span><span class="p">,</span> <span class="n">context</span><span class="o">.</span><span class="n">get_current_time</span><span class="p">()</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">delay</span><span class="p">)</span>
<span class="n">context</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">STATE_TAG</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterProcessingTime.on_merge"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.on_merge">[docs]</a> <span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># timers will be kept through merging</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="AfterProcessingTime.should_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.should_fire">[docs]</a> <span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">if</span> <span class="n">time_domain</span> <span class="o">==</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">REAL_TIME</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">True</span></div>
<div class="viewcode-block" id="AfterProcessingTime.on_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.on_fire">[docs]</a> <span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span></div>
<div class="viewcode-block" id="AfterProcessingTime.reset"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.reset">[docs]</a> <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="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">STATE_TAG</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterProcessingTime.may_lose_data"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.may_lose_data">[docs]</a> <span class="k">def</span> <span class="nf">may_lose_data</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_windowing</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;AfterProcessingTime may finish.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">MAY_FINISH</span></div>
<div class="viewcode-block" id="AfterProcessingTime.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.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="k">return</span> <span class="n">AfterProcessingTime</span><span class="p">(</span>
<span class="n">delay</span><span class="o">=</span><span class="p">(</span>
<span class="n">proto</span><span class="o">.</span><span class="n">after_processing_time</span><span class="o">.</span><span class="n">timestamp_transforms</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">delay</span><span class="o">.</span>
<span class="n">delay_millis</span><span class="p">)</span> <span class="o">//</span> <span class="mi">1000</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterProcessingTime.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.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="n">delay_proto</span> <span class="o">=</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">TimestampTransform</span><span class="p">(</span>
<span class="n">delay</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">TimestampTransform</span><span class="o">.</span><span class="n">Delay</span><span class="p">(</span>
<span class="n">delay_millis</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">delay</span> <span class="o">*</span> <span class="mi">1000</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">Trigger</span><span class="p">(</span>
<span class="n">after_processing_time</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">AfterProcessingTime</span><span class="p">(</span>
<span class="n">timestamp_transforms</span><span class="o">=</span><span class="p">[</span><span class="n">delay_proto</span><span class="p">]))</span></div>
<div class="viewcode-block" id="AfterProcessingTime.has_ontime_pane"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterProcessingTime.has_ontime_pane">[docs]</a> <span class="k">def</span> <span class="nf">has_ontime_pane</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span></div></div>
<span class="k">class</span> <span class="nc">Always</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Repeatedly invoke the given trigger, never finishing.&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="k">pass</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;Always&#39;</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">return</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">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="mi">1</span>
<span class="k">def</span> <span class="nf">on_element</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="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">has_ontime_pane</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</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="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">def</span> <span class="nf">may_lose_data</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_windowing</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;No potential loss, since the trigger always fires.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">Always</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="p">(</span>
<span class="n">always</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">Always</span><span class="p">())</span>
<span class="k">class</span> <span class="nc">_Never</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A trigger that never fires.</span>
<span class="sd"> Data may still be released at window closing.</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="k">pass</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;Never&#39;</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">return</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">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="nb">type</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">on_element</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="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">has_ontime_pane</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="kc">False</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="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">def</span> <span class="nf">may_lose_data</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_windowing</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;No potential data loss.</span>
<span class="sd"> Though Never doesn&#39;t explicitly trigger, it still collects data on</span>
<span class="sd"> windowing closing.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_Never</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="p">(</span>
<span class="n">never</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">Never</span><span class="p">())</span>
<div class="viewcode-block" id="AfterWatermark"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark">[docs]</a><span class="k">class</span> <span class="nc">AfterWatermark</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Fire exactly once when the watermark passes the end of the window.</span>
<span class="sd"> Args:</span>
<span class="sd"> early: if not None, a speculative trigger to repeatedly evaluate before</span>
<span class="sd"> the watermark passes the end of the window</span>
<span class="sd"> late: if not None, a speculative trigger to repeatedly evaluate after</span>
<span class="sd"> the watermark passes the end of the window</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">LATE_TAG</span> <span class="o">=</span> <span class="n">_CombiningValueStateTag</span><span class="p">(</span><span class="s1">&#39;is_late&#39;</span><span class="p">,</span> <span class="nb">any</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">early</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">late</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="c1"># TODO(zhoufek): Maybe don&#39;t wrap early/late if they are already Repeatedly</span>
<span class="bp">self</span><span class="o">.</span><span class="n">early</span> <span class="o">=</span> <span class="n">Repeatedly</span><span class="p">(</span><span class="n">early</span><span class="p">)</span> <span class="k">if</span> <span class="n">early</span> <span class="k">else</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">late</span> <span class="o">=</span> <span class="n">Repeatedly</span><span class="p">(</span><span class="n">late</span><span class="p">)</span> <span class="k">if</span> <span class="n">late</span> <span class="k">else</span> <span class="kc">None</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="n">qualifiers</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="p">:</span>
<span class="n">qualifiers</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="s1">&#39;early=</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">early</span><span class="o">.</span><span class="n">underlying</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="p">:</span>
<span class="n">qualifiers</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="s1">&#39;late=</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">late</span><span class="o">.</span><span class="n">underlying</span><span class="p">)</span>
<span class="k">return</span> <span class="s1">&#39;AfterWatermark(</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">qualifiers</span><span class="p">)</span>
<div class="viewcode-block" id="AfterWatermark.is_late"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.is_late">[docs]</a> <span class="k">def</span> <span class="nf">is_late</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span> <span class="ow">and</span> <span class="n">context</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">LATE_TAG</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterWatermark.on_element"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.on_element">[docs]</a> <span class="k">def</span> <span class="nf">on_element</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="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">is_late</span><span class="p">(</span><span class="n">context</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="o">.</span><span class="n">on_element</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;late&#39;</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">context</span><span class="o">.</span><span class="n">set_timer</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">window</span><span class="o">.</span><span class="n">end</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="o">.</span><span class="n">on_element</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;early&#39;</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterWatermark.on_merge"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.on_merge">[docs]</a> <span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># TODO(robertwb): Figure out whether the &#39;rewind&#39; semantics could be used</span>
<span class="c1"># here.</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_late</span><span class="p">(</span><span class="n">context</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="o">.</span><span class="n">on_merge</span><span class="p">(</span>
<span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;late&#39;</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Note: Timer clearing solely an optimization.</span>
<span class="k">for</span> <span class="n">window</span> <span class="ow">in</span> <span class="n">to_be_merged</span><span class="p">:</span>
<span class="k">if</span> <span class="n">window</span><span class="o">.</span><span class="n">end</span> <span class="o">!=</span> <span class="n">merge_result</span><span class="o">.</span><span class="n">end</span><span class="p">:</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_timer</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="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="o">.</span><span class="n">on_merge</span><span class="p">(</span>
<span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;early&#39;</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterWatermark.should_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.should_fire">[docs]</a> <span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</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">is_late</span><span class="p">(</span><span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="o">.</span><span class="n">should_fire</span><span class="p">(</span>
<span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;late&#39;</span><span class="p">))</span>
<span class="k">elif</span> <span class="n">watermark</span> <span class="o">&gt;=</span> <span class="n">window</span><span class="o">.</span><span class="n">end</span><span class="p">:</span>
<span class="c1"># Explicitly clear the timer so that late elements are not emitted again</span>
<span class="c1"># when the timer is fired.</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_timer</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="k">return</span> <span class="kc">True</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="o">.</span><span class="n">should_fire</span><span class="p">(</span>
<span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;early&#39;</span><span class="p">))</span>
<span class="k">return</span> <span class="kc">False</span></div>
<div class="viewcode-block" id="AfterWatermark.on_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.on_fire">[docs]</a> <span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</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">is_late</span><span class="p">(</span><span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="o">.</span><span class="n">on_fire</span><span class="p">(</span>
<span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;late&#39;</span><span class="p">))</span>
<span class="k">elif</span> <span class="n">watermark</span> <span class="o">&gt;=</span> <span class="n">window</span><span class="o">.</span><span class="n">end</span><span class="p">:</span>
<span class="n">context</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">LATE_TAG</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span>
<span class="k">return</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="o">.</span><span class="n">on_fire</span><span class="p">(</span><span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;early&#39;</span><span class="p">))</span>
<span class="k">return</span> <span class="kc">False</span></div>
<div class="viewcode-block" id="AfterWatermark.reset"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.reset">[docs]</a> <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="n">window</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">late</span><span class="p">:</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">LATE_TAG</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="o">.</span><span class="n">reset</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;early&#39;</span><span class="p">))</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="o">.</span><span class="n">reset</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;late&#39;</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterWatermark.may_lose_data"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.may_lose_data">[docs]</a> <span class="k">def</span> <span class="nf">may_lose_data</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="w"> </span><span class="sd">&quot;&quot;&quot;May cause data loss if lateness allowed and no late trigger set.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">windowing</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">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">MAY_FINISH</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</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></div>
<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">return</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="o">==</span> <span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">early</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">late</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">late</span><span class="p">)</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="nb">type</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">early</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="p">))</span>
<div class="viewcode-block" id="AfterWatermark.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.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="k">return</span> <span class="n">AfterWatermark</span><span class="p">(</span>
<span class="n">early</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">after_end_of_window</span><span class="o">.</span><span class="n">early_firings</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span>
<span class="k">if</span> <span class="n">proto</span><span class="o">.</span><span class="n">after_end_of_window</span><span class="o">.</span><span class="n">HasField</span><span class="p">(</span><span class="s1">&#39;early_firings&#39;</span><span class="p">)</span> <span class="k">else</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">late</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">after_end_of_window</span><span class="o">.</span><span class="n">late_firings</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span>
<span class="k">if</span> <span class="n">proto</span><span class="o">.</span><span class="n">after_end_of_window</span><span class="o">.</span><span class="n">HasField</span><span class="p">(</span><span class="s1">&#39;late_firings&#39;</span><span class="p">)</span> <span class="k">else</span> <span class="kc">None</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterWatermark.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.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="n">early_proto</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span><span class="o">.</span><span class="n">underlying</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">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">early</span> <span class="k">else</span> <span class="kc">None</span>
<span class="n">late_proto</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span><span class="o">.</span><span class="n">underlying</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">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">late</span> <span class="k">else</span> <span class="kc">None</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="p">(</span>
<span class="n">after_end_of_window</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">AfterEndOfWindow</span><span class="p">(</span>
<span class="n">early_firings</span><span class="o">=</span><span class="n">early_proto</span><span class="p">,</span> <span class="n">late_firings</span><span class="o">=</span><span class="n">late_proto</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterWatermark.has_ontime_pane"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterWatermark.has_ontime_pane">[docs]</a> <span class="k">def</span> <span class="nf">has_ontime_pane</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>
<div class="viewcode-block" id="AfterCount"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount">[docs]</a><span class="k">class</span> <span class="nc">AfterCount</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Fire when there are at least count elements in this window pane.</span>
<span class="sd"> AfterCount is experimental. No backwards compatibility guarantees.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">COUNT_TAG</span> <span class="o">=</span> <span class="n">_CombiningValueStateTag</span><span class="p">(</span><span class="s1">&#39;count&#39;</span><span class="p">,</span> <span class="n">combiners</span><span class="o">.</span><span class="n">CountCombineFn</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">count</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">count</span><span class="p">,</span> <span class="n">numbers</span><span class="o">.</span><span class="n">Integral</span><span class="p">)</span> <span class="ow">or</span> <span class="n">count</span> <span class="o">&lt;</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="s2">&quot;count (</span><span class="si">%d</span><span class="s2">) must be a positive integer.&quot;</span> <span class="o">%</span> <span class="n">count</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">count</span> <span class="o">=</span> <span class="n">count</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;AfterCount(</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">count</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">return</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="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">count</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">count</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">count</span><span class="p">)</span>
<div class="viewcode-block" id="AfterCount.on_element"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.on_element">[docs]</a> <span class="k">def</span> <span class="nf">on_element</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="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">context</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">COUNT_TAG</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterCount.on_merge"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.on_merge">[docs]</a> <span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># states automatically merged</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="AfterCount.should_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.should_fire">[docs]</a> <span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">context</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">COUNT_TAG</span><span class="p">)</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">count</span></div>
<div class="viewcode-block" id="AfterCount.on_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.on_fire">[docs]</a> <span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span></div>
<div class="viewcode-block" id="AfterCount.reset"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.reset">[docs]</a> <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="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">COUNT_TAG</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterCount.may_lose_data"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.may_lose_data">[docs]</a> <span class="k">def</span> <span class="nf">may_lose_data</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_windowing</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;AfterCount may finish.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">MAY_FINISH</span></div>
<div class="viewcode-block" id="AfterCount.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.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">unused_context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">AfterCount</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">element_count</span><span class="o">.</span><span class="n">element_count</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterCount.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.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">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">Trigger</span><span class="p">(</span>
<span class="n">element_count</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">ElementCount</span><span class="p">(</span>
<span class="n">element_count</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">count</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterCount.has_ontime_pane"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterCount.has_ontime_pane">[docs]</a> <span class="k">def</span> <span class="nf">has_ontime_pane</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span></div></div>
<div class="viewcode-block" id="Repeatedly"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly">[docs]</a><span class="k">class</span> <span class="nc">Repeatedly</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Repeatedly invoke the given trigger, never finishing.&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">underlying</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">underlying</span> <span class="o">=</span> <span class="n">underlying</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;Repeatedly(</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">underlying</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">return</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="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">underlying</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">underlying</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">underlying</span><span class="p">)</span>
<div class="viewcode-block" id="Repeatedly.on_element"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.on_element">[docs]</a> <span class="k">def</span> <span class="nf">on_element</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="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">underlying</span><span class="o">.</span><span class="n">on_element</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span></div>
<div class="viewcode-block" id="Repeatedly.on_merge"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.on_merge">[docs]</a> <span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">underlying</span><span class="o">.</span><span class="n">on_merge</span><span class="p">(</span><span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span></div>
<div class="viewcode-block" id="Repeatedly.should_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.should_fire">[docs]</a> <span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">underlying</span><span class="o">.</span><span class="n">should_fire</span><span class="p">(</span><span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span></div>
<div class="viewcode-block" id="Repeatedly.on_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.on_fire">[docs]</a> <span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</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">underlying</span><span class="o">.</span><span class="n">on_fire</span><span class="p">(</span><span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">underlying</span><span class="o">.</span><span class="n">reset</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">False</span></div>
<div class="viewcode-block" id="Repeatedly.reset"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.reset">[docs]</a> <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="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">underlying</span><span class="o">.</span><span class="n">reset</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span></div>
<div class="viewcode-block" id="Repeatedly.may_lose_data"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.may_lose_data">[docs]</a> <span class="k">def</span> <span class="nf">may_lose_data</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="w"> </span><span class="sd">&quot;&quot;&quot;Repeatedly will run in a loop and pick up whatever is left at GC.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span></div>
<div class="viewcode-block" id="Repeatedly.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.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="k">return</span> <span class="n">Repeatedly</span><span class="p">(</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">repeat</span><span class="o">.</span><span class="n">subtrigger</span><span class="p">,</span> <span class="n">context</span><span class="p">))</span></div>
<div class="viewcode-block" id="Repeatedly.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.to_runner_api">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="p">(</span>
<span class="n">repeat</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">Repeat</span><span class="p">(</span>
<span class="n">subtrigger</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">underlying</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="Repeatedly.has_ontime_pane"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.Repeatedly.has_ontime_pane">[docs]</a> <span class="k">def</span> <span class="nf">has_ontime_pane</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">underlying</span><span class="o">.</span><span class="n">has_ontime_pane</span><span class="p">()</span></div></div>
<span class="k">class</span> <span class="nc">_ParallelTriggerFn</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">,</span> <span class="n">metaclass</span><span class="o">=</span><span class="n">ABCMeta</span><span class="p">):</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">triggers</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggers</span> <span class="o">=</span> <span class="n">triggers</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;</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="s1">&#39;, &#39;</span><span class="o">.</span><span class="n">join</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="bp">self</span><span class="o">.</span><span class="n">triggers</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">return</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="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">triggers</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">triggers</span><span class="p">)</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">combine_op</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">trigger_results</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">on_element</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="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">trigger</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">triggers</span><span class="p">):</span>
<span class="n">trigger</span><span class="o">.</span><span class="n">on_element</span><span class="p">(</span><span class="n">element</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">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</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">trigger</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">triggers</span><span class="p">):</span>
<span class="n">trigger</span><span class="o">.</span><span class="n">on_merge</span><span class="p">(</span>
<span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_time_domain</span> <span class="o">=</span> <span class="n">time_domain</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_op</span><span class="p">(</span>
<span class="n">trigger</span><span class="o">.</span><span class="n">should_fire</span><span class="p">(</span>
<span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</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">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span>
<span class="k">for</span> <span class="n">ix</span><span class="p">,</span>
<span class="n">trigger</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">triggers</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">finished</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">trigger</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">triggers</span><span class="p">):</span>
<span class="n">nested_context</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">)</span>
<span class="k">if</span> <span class="n">trigger</span><span class="o">.</span><span class="n">should_fire</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">watermark</span><span class="p">,</span>
<span class="n">window</span><span class="p">,</span>
<span class="n">nested_context</span><span class="p">):</span>
<span class="n">finished</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">trigger</span><span class="o">.</span><span class="n">on_fire</span><span class="p">(</span><span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">nested_context</span><span class="p">))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_op</span><span class="p">(</span><span class="n">finished</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">may_lose_data</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="n">may_finish</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_op</span><span class="p">(</span>
<span class="n">_IncludesMayFinish</span><span class="p">(</span><span class="n">t</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">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">)</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">DataLossReason</span><span class="o">.</span><span class="n">MAY_FINISH</span>
<span class="k">if</span> <span class="n">may_finish</span> <span class="k">else</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span><span class="p">)</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="n">window</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">trigger</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">triggers</span><span class="p">):</span>
<span class="n">trigger</span><span class="o">.</span><span class="n">reset</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">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">index</span><span class="p">):</span>
<span class="k">return</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;</span><span class="si">%d</span><span class="s1">/&#39;</span> <span class="o">%</span> <span class="n">index</span><span class="p">)</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api</span><span class="p">(</span><span class="n">proto</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">subtriggers</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">TriggerFn</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">subtrigger</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span> <span class="k">for</span> <span class="n">subtrigger</span> <span class="ow">in</span>
<span class="n">proto</span><span class="o">.</span><span class="n">after_all</span><span class="o">.</span><span class="n">subtriggers</span> <span class="ow">or</span> <span class="n">proto</span><span class="o">.</span><span class="n">after_any</span><span class="o">.</span><span class="n">subtriggers</span>
<span class="p">]</span>
<span class="k">if</span> <span class="n">proto</span><span class="o">.</span><span class="n">after_all</span><span class="o">.</span><span class="n">subtriggers</span><span class="p">:</span>
<span class="k">return</span> <span class="n">AfterAll</span><span class="p">(</span><span class="o">*</span><span class="n">subtriggers</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">AfterAny</span><span class="p">(</span><span class="o">*</span><span class="n">subtriggers</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">subtriggers</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">subtrigger</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">subtrigger</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span>
<span class="p">]</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_op</span> <span class="o">==</span> <span class="nb">all</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">Trigger</span><span class="p">(</span>
<span class="n">after_all</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">AfterAll</span><span class="p">(</span>
<span class="n">subtriggers</span><span class="o">=</span><span class="n">subtriggers</span><span class="p">))</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">combine_op</span> <span class="o">==</span> <span class="nb">any</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">Trigger</span><span class="p">(</span>
<span class="n">after_any</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">AfterAny</span><span class="p">(</span>
<span class="n">subtriggers</span><span class="o">=</span><span class="n">subtriggers</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">has_ontime_pane</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">any</span><span class="p">(</span><span class="n">t</span><span class="o">.</span><span class="n">has_ontime_pane</span><span class="p">()</span> <span class="k">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">)</span>
<div class="viewcode-block" id="AfterAny"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterAny">[docs]</a><span class="k">class</span> <span class="nc">AfterAny</span><span class="p">(</span><span class="n">_ParallelTriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Fires when any subtrigger fires.</span>
<span class="sd"> Also finishes when any subtrigger finishes.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">combine_op</span> <span class="o">=</span> <span class="nb">any</span></div>
<div class="viewcode-block" id="AfterAll"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterAll">[docs]</a><span class="k">class</span> <span class="nc">AfterAll</span><span class="p">(</span><span class="n">_ParallelTriggerFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Fires when all subtriggers have fired.</span>
<span class="sd"> Also finishes when all subtriggers have finished.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">combine_op</span> <span class="o">=</span> <span class="nb">all</span></div>
<div class="viewcode-block" id="AfterEach"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach">[docs]</a><span class="k">class</span> <span class="nc">AfterEach</span><span class="p">(</span><span class="n">TriggerFn</span><span class="p">):</span>
<span class="n">INDEX_TAG</span> <span class="o">=</span> <span class="n">_CombiningValueStateTag</span><span class="p">(</span>
<span class="s1">&#39;index&#39;</span><span class="p">,</span> <span class="p">(</span><span class="k">lambda</span> <span class="n">indices</span><span class="p">:</span> <span class="mi">0</span> <span class="k">if</span> <span class="ow">not</span> <span class="n">indices</span> <span class="k">else</span> <span class="nb">max</span><span class="p">(</span><span class="n">indices</span><span class="p">)))</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">triggers</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggers</span> <span class="o">=</span> <span class="n">triggers</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;</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="s1">&#39;, &#39;</span><span class="o">.</span><span class="n">join</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="bp">self</span><span class="o">.</span><span class="n">triggers</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">return</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="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">triggers</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">triggers</span><span class="p">)</span>
<div class="viewcode-block" id="AfterEach.on_element"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.on_element">[docs]</a> <span class="k">def</span> <span class="nf">on_element</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="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">ix</span> <span class="o">=</span> <span class="n">context</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">INDEX_TAG</span><span class="p">)</span>
<span class="k">if</span> <span class="n">ix</span> <span class="o">&lt;</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">[</span><span class="n">ix</span><span class="p">]</span><span class="o">.</span><span class="n">on_element</span><span class="p">(</span>
<span class="n">element</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">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterEach.on_merge"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.on_merge">[docs]</a> <span class="k">def</span> <span class="nf">on_merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># This takes the furthest window on merging.</span>
<span class="c1"># TODO(robertwb): Revisit this when merging windows logic is settled for</span>
<span class="c1"># all possible merging situations.</span>
<span class="n">ix</span> <span class="o">=</span> <span class="n">context</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">INDEX_TAG</span><span class="p">)</span>
<span class="k">if</span> <span class="n">ix</span> <span class="o">&lt;</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">[</span><span class="n">ix</span><span class="p">]</span><span class="o">.</span><span class="n">on_merge</span><span class="p">(</span>
<span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterEach.should_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.should_fire">[docs]</a> <span class="k">def</span> <span class="nf">should_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">ix</span> <span class="o">=</span> <span class="n">context</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">INDEX_TAG</span><span class="p">)</span>
<span class="k">if</span> <span class="n">ix</span> <span class="o">&lt;</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">[</span><span class="n">ix</span><span class="p">]</span><span class="o">.</span><span class="n">should_fire</span><span class="p">(</span>
<span class="n">time_domain</span><span class="p">,</span> <span class="n">watermark</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">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterEach.on_fire"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.on_fire">[docs]</a> <span class="k">def</span> <span class="nf">on_fire</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">ix</span> <span class="o">=</span> <span class="n">context</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">INDEX_TAG</span><span class="p">)</span>
<span class="k">if</span> <span class="n">ix</span> <span class="o">&lt;</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">[</span><span class="n">ix</span><span class="p">]</span><span class="o">.</span><span class="n">on_fire</span><span class="p">(</span><span class="n">watermark</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">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">)):</span>
<span class="n">ix</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="n">context</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">INDEX_TAG</span><span class="p">,</span> <span class="n">ix</span><span class="p">)</span>
<span class="k">return</span> <span class="n">ix</span> <span class="o">==</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">)</span></div>
<div class="viewcode-block" id="AfterEach.reset"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.reset">[docs]</a> <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="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">context</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">INDEX_TAG</span><span class="p">)</span>
<span class="k">for</span> <span class="n">ix</span><span class="p">,</span> <span class="n">trigger</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">triggers</span><span class="p">):</span>
<span class="n">trigger</span><span class="o">.</span><span class="n">reset</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">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">ix</span><span class="p">))</span></div>
<div class="viewcode-block" id="AfterEach.may_lose_data"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.may_lose_data">[docs]</a> <span class="k">def</span> <span class="nf">may_lose_data</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="w"> </span><span class="sd">&quot;&quot;&quot;If all sub-triggers may finish, this may finish.&quot;&quot;&quot;</span>
<span class="n">may_finish</span> <span class="o">=</span> <span class="nb">all</span><span class="p">(</span>
<span class="n">_IncludesMayFinish</span><span class="p">(</span><span class="n">t</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">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">)</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">DataLossReason</span><span class="o">.</span><span class="n">MAY_FINISH</span>
<span class="k">if</span> <span class="n">may_finish</span> <span class="k">else</span> <span class="n">DataLossReason</span><span class="o">.</span><span class="n">NO_POTENTIAL_LOSS</span><span class="p">)</span></div>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_sub_context</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="n">index</span><span class="p">):</span>
<span class="k">return</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">context</span><span class="p">,</span> <span class="s1">&#39;</span><span class="si">%d</span><span class="s1">/&#39;</span> <span class="o">%</span> <span class="n">index</span><span class="p">)</span>
<div class="viewcode-block" id="AfterEach.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.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="k">return</span> <span class="n">AfterEach</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">TriggerFn</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span><span class="n">subtrigger</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span>
<span class="k">for</span> <span class="n">subtrigger</span> <span class="ow">in</span> <span class="n">proto</span><span class="o">.</span><span class="n">after_each</span><span class="o">.</span><span class="n">subtriggers</span>
<span class="p">])</span></div>
<div class="viewcode-block" id="AfterEach.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.to_runner_api">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="p">(</span>
<span class="n">after_each</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">AfterEach</span><span class="p">(</span>
<span class="n">subtriggers</span><span class="o">=</span><span class="p">[</span>
<span class="n">subtrigger</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">subtrigger</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span>
<span class="p">]))</span></div>
<div class="viewcode-block" id="AfterEach.has_ontime_pane"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.AfterEach.has_ontime_pane">[docs]</a> <span class="k">def</span> <span class="nf">has_ontime_pane</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">any</span><span class="p">(</span><span class="n">t</span><span class="o">.</span><span class="n">has_ontime_pane</span><span class="p">()</span> <span class="k">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="OrFinally"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.OrFinally">[docs]</a><span class="k">class</span> <span class="nc">OrFinally</span><span class="p">(</span><span class="n">AfterAny</span><span class="p">):</span>
<div class="viewcode-block" id="OrFinally.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.OrFinally.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="k">return</span> <span class="n">OrFinally</span><span class="p">(</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">or_finally</span><span class="o">.</span><span class="n">main</span><span class="p">,</span> <span class="n">context</span><span class="p">),</span>
<span class="c1"># getattr is used as finally is a keyword in Python</span>
<span class="n">TriggerFn</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span>
<span class="nb">getattr</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">or_finally</span><span class="p">,</span> <span class="s1">&#39;finally&#39;</span><span class="p">),</span> <span class="n">context</span><span class="p">))</span></div>
<div class="viewcode-block" id="OrFinally.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.trigger.html#apache_beam.transforms.trigger.OrFinally.to_runner_api">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="p">(</span>
<span class="n">or_finally</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Trigger</span><span class="o">.</span><span class="n">OrFinally</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">triggers</span><span class="p">[</span><span class="mi">0</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="c1"># dict keyword argument is used as finally is a keyword in Python</span>
<span class="o">**</span><span class="p">{</span><span class="s1">&#39;finally&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">triggers</span><span class="p">[</span><span class="mi">1</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></div></div>
<span class="k">class</span> <span class="nc">TriggerContext</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">outer</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">clock</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span> <span class="o">=</span> <span class="n">outer</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_window</span> <span class="o">=</span> <span class="n">window</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_clock</span> <span class="o">=</span> <span class="n">clock</span>
<span class="k">def</span> <span class="nf">get_current_time</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">_clock</span><span class="o">.</span><span class="n">time</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">set_timer</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="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">set_timer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_window</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">clear_timer</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="n">time_domain</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">clear_timer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_window</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">add_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_window</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_window</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_window</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">NestedContext</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Namespaced context useful for defining composite triggers.&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">outer</span><span class="p">,</span> <span class="n">prefix</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span> <span class="o">=</span> <span class="n">outer</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_prefix</span> <span class="o">=</span> <span class="n">prefix</span>
<span class="k">def</span> <span class="nf">get_current_time</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">_outer</span><span class="o">.</span><span class="n">get_current_time</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">set_timer</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="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">set_timer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_prefix</span> <span class="o">+</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">clear_timer</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="n">time_domain</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">clear_timer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_prefix</span> <span class="o">+</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">add_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span><span class="n">tag</span><span class="o">.</span><span class="n">with_prefix</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_prefix</span><span class="p">),</span> <span class="n">value</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="n">tag</span><span class="o">.</span><span class="n">with_prefix</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_prefix</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_outer</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="n">tag</span><span class="o">.</span><span class="n">with_prefix</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_prefix</span><span class="p">))</span>
<span class="c1"># pylint: disable=unused-argument</span>
<span class="k">class</span> <span class="nc">SimpleState</span><span class="p">(</span><span class="n">metaclass</span><span class="o">=</span><span class="n">ABCMeta</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Basic state storage interface used for triggering.</span>
<span class="sd"> Only timers must hold the watermark (by their timestamp).</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">set_timer</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">):</span>
<span class="k">pass</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">get_window</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window_id</span><span class="p">):</span>
<span class="k">pass</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">clear_timer</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">):</span>
<span class="k">pass</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">add_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="k">pass</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">get_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="k">pass</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">at</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">clock</span><span class="p">):</span>
<span class="k">return</span> <span class="n">NestedContext</span><span class="p">(</span><span class="n">TriggerContext</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">clock</span><span class="p">),</span> <span class="s1">&#39;trigger&#39;</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">UnmergedState</span><span class="p">(</span><span class="n">SimpleState</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;State suitable for use in TriggerDriver.</span>
<span class="sd"> This class must be implemented by each backend.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">set_global_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="k">pass</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">get_global_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">default</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">pass</span>
<span class="c1"># pylint: enable=unused-argument</span>
<span class="k">class</span> <span class="nc">MergeableStateAdapter</span><span class="p">(</span><span class="n">SimpleState</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Wraps an UnmergedState, tracking merged windows.&quot;&quot;&quot;</span>
<span class="c1"># TODO(robertwb): A similar indirection could be used for sliding windows</span>
<span class="c1"># or other window_fns when a single element typically belongs to many windows.</span>
<span class="n">WINDOW_IDS</span> <span class="o">=</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">(</span><span class="s1">&#39;window_ids&#39;</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">raw_state</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">raw_state</span> <span class="o">=</span> <span class="n">raw_state</span>
<span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">raw_state</span><span class="o">.</span><span class="n">get_global_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">WINDOW_IDS</span><span class="p">,</span> <span class="p">{})</span>
<span class="bp">self</span><span class="o">.</span><span class="n">counter</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">set_timer</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">raw_state</span><span class="o">.</span><span class="n">set_timer</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_get_id</span><span class="p">(</span><span class="n">window</span><span class="p">),</span>
<span class="n">name</span><span class="p">,</span>
<span class="n">time_domain</span><span class="p">,</span>
<span class="n">timestamp</span><span class="p">,</span>
<span class="n">dynamic_timer_tag</span><span class="o">=</span><span class="n">dynamic_timer_tag</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">clear_timer</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">):</span>
<span class="k">for</span> <span class="n">window_id</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_ids</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">raw_state</span><span class="o">.</span><span class="n">clear_timer</span><span class="p">(</span>
<span class="n">window_id</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="o">=</span><span class="n">dynamic_timer_tag</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">add_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Merging requested for non-mergeable state tag: </span><span class="si">%r</span><span class="s1">.&#39;</span> <span class="o">%</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_CombiningValueStateTag</span><span class="p">):</span>
<span class="n">tag</span> <span class="o">=</span> <span class="n">tag</span><span class="o">.</span><span class="n">without_extraction</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">raw_state</span><span class="o">.</span><span class="n">add_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_get_id</span><span class="p">(</span><span class="n">window</span><span class="p">),</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_CombiningValueStateTag</span><span class="p">):</span>
<span class="n">original_tag</span><span class="p">,</span> <span class="n">tag</span> <span class="o">=</span> <span class="n">tag</span><span class="p">,</span> <span class="n">tag</span><span class="o">.</span><span class="n">without_extraction</span><span class="p">()</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">raw_state</span><span class="o">.</span><span class="n">get_state</span><span class="p">(</span><span class="n">window_id</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">for</span> <span class="n">window_id</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_ids</span><span class="p">(</span><span class="n">window</span><span class="p">)</span>
<span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Merging requested for non-mergeable state tag: </span><span class="si">%r</span><span class="s1">.&#39;</span> <span class="o">%</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_CombiningValueStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="n">original_tag</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">extract_output</span><span class="p">(</span>
<span class="n">original_tag</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">merge_accumulators</span><span class="p">(</span><span class="n">values</span><span class="p">))</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_ListStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span><span class="n">v</span> <span class="k">for</span> <span class="n">vs</span> <span class="ow">in</span> <span class="n">values</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">vs</span><span class="p">]</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_SetStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="p">{</span><span class="n">v</span> <span class="k">for</span> <span class="n">vs</span> <span class="ow">in</span> <span class="n">values</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">vs</span><span class="p">}</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_WatermarkHoldStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="n">tag</span><span class="o">.</span><span class="n">timestamp_combiner_impl</span><span class="o">.</span><span class="n">combine_all</span><span class="p">(</span><span class="n">values</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;Invalid tag.&#39;</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="k">for</span> <span class="n">window_id</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_ids</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">raw_state</span><span class="o">.</span><span class="n">clear_state</span><span class="p">(</span><span class="n">window_id</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">if</span> <span class="n">tag</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</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">_persist_window_ids</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">merge</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">):</span>
<span class="k">for</span> <span class="n">window</span> <span class="ow">in</span> <span class="n">to_be_merged</span><span class="p">:</span>
<span class="k">if</span> <span class="n">window</span> <span class="o">!=</span> <span class="n">merge_result</span><span class="p">:</span>
<span class="k">if</span> <span class="n">window</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">:</span>
<span class="k">if</span> <span class="n">merge_result</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">:</span>
<span class="n">merge_window_ids</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">[</span><span class="n">merge_result</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">merge_window_ids</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">[</span><span class="n">merge_result</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">merge_window_ids</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="o">.</span><span class="n">pop</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">_persist_window_ids</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">known_windows</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_window</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window_id</span><span class="p">):</span>
<span class="k">for</span> <span class="n">window</span><span class="p">,</span> <span class="n">ids</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">if</span> <span class="n">window_id</span> <span class="ow">in</span> <span class="n">ids</span><span class="p">:</span>
<span class="k">return</span> <span class="n">window</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;No window for </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">window_id</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_get_id</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">):</span>
<span class="k">if</span> <span class="n">window</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="mi">0</span><span class="p">]</span>
<span class="n">window_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_next_counter</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">[</span><span class="n">window</span><span class="p">]</span> <span class="o">=</span> <span class="p">[</span><span class="n">window_id</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_persist_window_ids</span><span class="p">()</span>
<span class="k">return</span> <span class="n">window_id</span>
<span class="k">def</span> <span class="nf">_get_ids</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="p">[])</span>
<span class="k">def</span> <span class="nf">_get_next_counter</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">window_ids</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">counter</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">counter</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">counter</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="n">k</span> <span class="k">for</span> <span class="n">ids</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="o">.</span><span class="n">values</span><span class="p">()</span> <span class="k">for</span> <span class="n">k</span> <span class="ow">in</span> <span class="n">ids</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">counter</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">counter</span>
<span class="k">def</span> <span class="nf">_persist_window_ids</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">raw_state</span><span class="o">.</span><span class="n">set_global_state</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">WINDOW_IDS</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">window_ids</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;</span><span class="se">\n\t</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">([</span><span class="nb">repr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">window_ids</span><span class="p">)]</span> <span class="o">+</span>
<span class="nb">repr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">raw_state</span><span class="p">)</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">create_trigger_driver</span><span class="p">(</span>
<span class="n">windowing</span><span class="p">,</span> <span class="n">is_batch</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">phased_combine_fn</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">clock</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Create the TriggerDriver for the given windowing and options.&quot;&quot;&quot;</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/20165): Respect closing and</span>
<span class="c1"># on-time behaviors. For batch, we should always fire once, no matter what.</span>
<span class="k">if</span> <span class="n">is_batch</span> <span class="ow">and</span> <span class="n">windowing</span><span class="o">.</span><span class="n">triggerfn</span> <span class="o">==</span> <span class="n">_Never</span><span class="p">():</span>
<span class="n">windowing</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="n">windowing</span><span class="p">)</span>
<span class="n">windowing</span><span class="o">.</span><span class="n">triggerfn</span> <span class="o">=</span> <span class="n">Always</span><span class="p">()</span>
<span class="c1"># TODO(robertwb): We can do more if we know elements are in timestamp</span>
<span class="c1"># sorted order.</span>
<span class="k">if</span> <span class="n">windowing</span><span class="o">.</span><span class="n">is_default</span><span class="p">()</span> <span class="ow">and</span> <span class="n">is_batch</span><span class="p">:</span>
<span class="n">driver</span> <span class="o">=</span> <span class="n">BatchGlobalTriggerDriver</span><span class="p">()</span>
<span class="k">elif</span> <span class="p">(</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="ow">and</span>
<span class="p">(</span><span class="n">windowing</span><span class="o">.</span><span class="n">triggerfn</span> <span class="ow">in</span> <span class="p">[</span><span class="n">AfterCount</span><span class="p">(</span><span class="mi">1</span><span class="p">),</span> <span class="n">Always</span><span class="p">()])</span> <span class="ow">and</span> <span class="n">is_batch</span><span class="p">):</span>
<span class="c1"># Here we also just pass through all the values exactly once.</span>
<span class="n">driver</span> <span class="o">=</span> <span class="n">BatchGlobalTriggerDriver</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">driver</span> <span class="o">=</span> <span class="n">GeneralTriggerDriver</span><span class="p">(</span><span class="n">windowing</span><span class="p">,</span> <span class="n">clock</span><span class="p">)</span>
<span class="k">if</span> <span class="n">phased_combine_fn</span><span class="p">:</span>
<span class="c1"># TODO(ccy): Refactor GeneralTriggerDriver to combine values eagerly using</span>
<span class="c1"># the known phased_combine_fn here.</span>
<span class="n">driver</span> <span class="o">=</span> <span class="n">CombiningTriggerDriver</span><span class="p">(</span><span class="n">phased_combine_fn</span><span class="p">,</span> <span class="n">driver</span><span class="p">)</span>
<span class="k">return</span> <span class="n">driver</span>
<span class="k">class</span> <span class="nc">TriggerDriver</span><span class="p">(</span><span class="n">metaclass</span><span class="o">=</span><span class="n">ABCMeta</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Breaks a series of bundle and timer firings into window (pane)s.&quot;&quot;&quot;</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">process_elements</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">windowed_values</span><span class="p">,</span>
<span class="n">output_watermark</span><span class="p">,</span>
<span class="n">input_watermark</span><span class="o">=</span><span class="n">MIN_TIMESTAMP</span><span class="p">):</span>
<span class="k">pass</span>
<span class="nd">@abstractmethod</span>
<span class="k">def</span> <span class="nf">process_timer</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">window_id</span><span class="p">,</span>
<span class="n">name</span><span class="p">,</span>
<span class="n">time_domain</span><span class="p">,</span>
<span class="n">timestamp</span><span class="p">,</span>
<span class="n">state</span><span class="p">,</span>
<span class="n">input_watermark</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">process_entire_key</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">windowed_values</span><span class="p">):</span>
<span class="c1"># This state holds per-key, multi-window state.</span>
<span class="n">state</span> <span class="o">=</span> <span class="n">InMemoryUnmergedState</span><span class="p">()</span>
<span class="k">for</span> <span class="n">wvalue</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">process_elements</span><span class="p">(</span><span class="n">state</span><span class="p">,</span>
<span class="n">windowed_values</span><span class="p">,</span>
<span class="n">MIN_TIMESTAMP</span><span class="p">,</span>
<span class="n">MIN_TIMESTAMP</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">wvalue</span><span class="o">.</span><span class="n">with_value</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">wvalue</span><span class="o">.</span><span class="n">value</span><span class="p">))</span>
<span class="k">while</span> <span class="n">state</span><span class="o">.</span><span class="n">timers</span><span class="p">:</span>
<span class="n">fired</span> <span class="o">=</span> <span class="n">state</span><span class="o">.</span><span class="n">get_and_clear_timers</span><span class="p">()</span>
<span class="k">for</span> <span class="n">timer_window</span><span class="p">,</span> <span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">fire_time</span><span class="p">,</span> <span class="n">_</span><span class="p">)</span> <span class="ow">in</span> <span class="n">fired</span><span class="p">:</span>
<span class="k">for</span> <span class="n">wvalue</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">process_timer</span><span class="p">(</span><span class="n">timer_window</span><span class="p">,</span>
<span class="n">name</span><span class="p">,</span>
<span class="n">time_domain</span><span class="p">,</span>
<span class="n">fire_time</span><span class="p">,</span>
<span class="n">state</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">wvalue</span><span class="o">.</span><span class="n">with_value</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">wvalue</span><span class="o">.</span><span class="n">value</span><span class="p">))</span>
<span class="k">class</span> <span class="nc">_UnwindowedValues</span><span class="p">(</span><span class="n">observable</span><span class="o">.</span><span class="n">ObservableMixin</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Exposes iterable of windowed values as iterable of unwindowed values.&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">windowed_values</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">_windowed_values</span> <span class="o">=</span> <span class="n">windowed_values</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">wv</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_windowed_values</span><span class="p">:</span>
<span class="n">unwindowed_value</span> <span class="o">=</span> <span class="n">wv</span><span class="o">.</span><span class="n">value</span>
<span class="bp">self</span><span class="o">.</span><span class="n">notify_observers</span><span class="p">(</span><span class="n">unwindowed_value</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">unwindowed_value</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;&lt;_UnwindowedValues of </span><span class="si">%s</span><span class="s1">&gt;&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_windowed_values</span>
<span class="k">def</span> <span class="nf">__reduce__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">,</span> <span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="p">),</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">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">collections_abc</span><span class="o">.</span><span class="n">Iterable</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">all</span><span class="p">(</span>
<span class="n">a</span> <span class="o">==</span> <span class="n">b</span> <span class="k">for</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span> <span class="ow">in</span> <span class="n">zip_longest</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="n">fillvalue</span><span class="o">=</span><span class="nb">object</span><span class="p">()))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">NotImplemented</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="nb">tuple</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span>
<span class="n">coder_impl</span><span class="o">.</span><span class="n">FastPrimitivesCoderImpl</span><span class="o">.</span><span class="n">register_iterable_like_type</span><span class="p">(</span>
<span class="n">_UnwindowedValues</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">BatchGlobalTriggerDriver</span><span class="p">(</span><span class="n">TriggerDriver</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Groups all received values together.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">GLOBAL_WINDOW_TUPLE</span> <span class="o">=</span> <span class="p">(</span><span class="n">GlobalWindow</span><span class="p">(),</span> <span class="p">)</span>
<span class="n">ONLY_FIRING</span> <span class="o">=</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">PaneInfo</span><span class="p">(</span>
<span class="n">is_first</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="n">is_last</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="n">timing</span><span class="o">=</span><span class="n">windowed_value</span><span class="o">.</span><span class="n">PaneInfoTiming</span><span class="o">.</span><span class="n">ON_TIME</span><span class="p">,</span>
<span class="n">index</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span>
<span class="n">nonspeculative_index</span><span class="o">=</span><span class="mi">0</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">process_elements</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">windowed_values</span><span class="p">,</span>
<span class="n">unused_output_watermark</span><span class="p">,</span>
<span class="n">unused_input_watermark</span><span class="o">=</span><span class="n">MIN_TIMESTAMP</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">WindowedValue</span><span class="p">(</span>
<span class="n">_UnwindowedValues</span><span class="p">(</span><span class="n">windowed_values</span><span class="p">),</span>
<span class="n">MIN_TIMESTAMP</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">GLOBAL_WINDOW_TUPLE</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ONLY_FIRING</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">process_timer</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">window_id</span><span class="p">,</span>
<span class="n">name</span><span class="p">,</span>
<span class="n">time_domain</span><span class="p">,</span>
<span class="n">timestamp</span><span class="p">,</span>
<span class="n">state</span><span class="p">,</span>
<span class="n">input_watermark</span><span class="o">=</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="s1">&#39;Triggers never set or called for batch default windowing.&#39;</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">CombiningTriggerDriver</span><span class="p">(</span><span class="n">TriggerDriver</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Uses a phased_combine_fn to process output of wrapped TriggerDriver.&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">phased_combine_fn</span><span class="p">,</span> <span class="n">underlying</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">phased_combine_fn</span> <span class="o">=</span> <span class="n">phased_combine_fn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">underlying</span> <span class="o">=</span> <span class="n">underlying</span>
<span class="k">def</span> <span class="nf">process_elements</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">windowed_values</span><span class="p">,</span>
<span class="n">output_watermark</span><span class="p">,</span>
<span class="n">input_watermark</span><span class="o">=</span><span class="n">MIN_TIMESTAMP</span><span class="p">):</span>
<span class="n">uncombined</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">underlying</span><span class="o">.</span><span class="n">process_elements</span><span class="p">(</span>
<span class="n">state</span><span class="p">,</span> <span class="n">windowed_values</span><span class="p">,</span> <span class="n">output_watermark</span><span class="p">,</span> <span class="n">input_watermark</span><span class="p">)</span>
<span class="k">for</span> <span class="n">output</span> <span class="ow">in</span> <span class="n">uncombined</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">output</span><span class="o">.</span><span class="n">with_value</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">phased_combine_fn</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">output</span><span class="o">.</span><span class="n">value</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">process_timer</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">window_id</span><span class="p">,</span>
<span class="n">name</span><span class="p">,</span>
<span class="n">time_domain</span><span class="p">,</span>
<span class="n">timestamp</span><span class="p">,</span>
<span class="n">state</span><span class="p">,</span>
<span class="n">input_watermark</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="n">uncombined</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">underlying</span><span class="o">.</span><span class="n">process_timer</span><span class="p">(</span>
<span class="n">window_id</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">state</span><span class="p">,</span> <span class="n">input_watermark</span><span class="p">)</span>
<span class="k">for</span> <span class="n">output</span> <span class="ow">in</span> <span class="n">uncombined</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">output</span><span class="o">.</span><span class="n">with_value</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">phased_combine_fn</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">output</span><span class="o">.</span><span class="n">value</span><span class="p">))</span>
<span class="k">class</span> <span class="nc">GeneralTriggerDriver</span><span class="p">(</span><span class="n">TriggerDriver</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Breaks a series of bundle and timer firings into window (pane)s.</span>
<span class="sd"> Suitable for all variants of Windowing.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">ELEMENTS</span> <span class="o">=</span> <span class="n">_ListStateTag</span><span class="p">(</span><span class="s1">&#39;elements&#39;</span><span class="p">)</span>
<span class="n">TOMBSTONE</span> <span class="o">=</span> <span class="n">_CombiningValueStateTag</span><span class="p">(</span><span class="s1">&#39;tombstone&#39;</span><span class="p">,</span> <span class="n">combiners</span><span class="o">.</span><span class="n">CountCombineFn</span><span class="p">())</span>
<span class="n">INDEX</span> <span class="o">=</span> <span class="n">_CombiningValueStateTag</span><span class="p">(</span><span class="s1">&#39;index&#39;</span><span class="p">,</span> <span class="n">combiners</span><span class="o">.</span><span class="n">CountCombineFn</span><span class="p">())</span>
<span class="n">NONSPECULATIVE_INDEX</span> <span class="o">=</span> <span class="n">_CombiningValueStateTag</span><span class="p">(</span>
<span class="s1">&#39;nonspeculative_index&#39;</span><span class="p">,</span> <span class="n">combiners</span><span class="o">.</span><span class="n">CountCombineFn</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">windowing</span><span class="p">,</span> <span class="n">clock</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">clock</span> <span class="o">=</span> <span class="n">clock</span>
<span class="bp">self</span><span class="o">.</span><span class="n">allowed_lateness</span> <span class="o">=</span> <span class="n">windowing</span><span class="o">.</span><span class="n">allowed_lateness</span>
<span class="bp">self</span><span class="o">.</span><span class="n">window_fn</span> <span class="o">=</span> <span class="n">windowing</span><span class="o">.</span><span class="n">windowfn</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner_impl</span> <span class="o">=</span> <span class="n">TimestampCombiner</span><span class="o">.</span><span class="n">get_impl</span><span class="p">(</span>
<span class="n">windowing</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">window_fn</span><span class="p">)</span>
<span class="c1"># pylint: disable=invalid-name</span>
<span class="bp">self</span><span class="o">.</span><span class="n">WATERMARK_HOLD</span> <span class="o">=</span> <span class="n">_WatermarkHoldStateTag</span><span class="p">(</span>
<span class="s1">&#39;watermark&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner_impl</span><span class="p">)</span>
<span class="c1"># pylint: enable=invalid-name</span>
<span class="bp">self</span><span class="o">.</span><span class="n">trigger_fn</span> <span class="o">=</span> <span class="n">windowing</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">windowing</span><span class="o">.</span><span class="n">accumulation_mode</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_merging</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">def</span> <span class="nf">process_elements</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">windowed_values</span><span class="p">,</span>
<span class="n">output_watermark</span><span class="p">,</span>
<span class="n">input_watermark</span><span class="o">=</span><span class="n">MIN_TIMESTAMP</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_merging</span><span class="p">:</span>
<span class="n">state</span> <span class="o">=</span> <span class="n">MergeableStateAdapter</span><span class="p">(</span><span class="n">state</span><span class="p">)</span>
<span class="n">windows_to_elements</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">)</span>
<span class="k">for</span> <span class="n">wv</span> <span class="ow">in</span> <span class="n">windowed_values</span><span class="p">:</span>
<span class="k">for</span> <span class="n">window</span> <span class="ow">in</span> <span class="n">wv</span><span class="o">.</span><span class="n">windows</span><span class="p">:</span>
<span class="c1"># ignore expired windows</span>
<span class="k">if</span> <span class="n">input_watermark</span> <span class="o">&gt;</span> <span class="n">window</span><span class="o">.</span><span class="n">end</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">allowed_lateness</span><span class="p">:</span>
<span class="k">continue</span>
<span class="n">windows_to_elements</span><span class="p">[</span><span class="n">window</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">wv</span><span class="o">.</span><span class="n">value</span><span class="p">,</span> <span class="n">wv</span><span class="o">.</span><span class="n">timestamp</span><span class="p">))</span>
<span class="c1"># First handle merging.</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_merging</span><span class="p">:</span>
<span class="n">old_windows</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">state</span><span class="o">.</span><span class="n">known_windows</span><span class="p">())</span>
<span class="n">all_windows</span> <span class="o">=</span> <span class="n">old_windows</span><span class="o">.</span><span class="n">union</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="n">windows_to_elements</span><span class="p">))</span>
<span class="k">if</span> <span class="n">all_windows</span> <span class="o">!=</span> <span class="n">old_windows</span><span class="p">:</span>
<span class="n">merged_away</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">class</span> <span class="nc">TriggerMergeContext</span><span class="p">(</span><span class="n">WindowFn</span><span class="o">.</span><span class="n">MergeContext</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">merge</span><span class="p">(</span><span class="n">_</span><span class="p">,</span> <span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">):</span> <span class="c1"># pylint: disable=no-self-argument</span>
<span class="k">for</span> <span class="n">window</span> <span class="ow">in</span> <span class="n">to_be_merged</span><span class="p">:</span>
<span class="k">if</span> <span class="n">window</span> <span class="o">!=</span> <span class="n">merge_result</span><span class="p">:</span>
<span class="n">merged_away</span><span class="p">[</span><span class="n">window</span><span class="p">]</span> <span class="o">=</span> <span class="n">merge_result</span>
<span class="c1"># Clear state associated with PaneInfo since it is</span>
<span class="c1"># not preserved across merges.</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">INDEX</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">NONSPECULATIVE_INDEX</span><span class="p">)</span>
<span class="n">state</span><span class="o">.</span><span class="n">merge</span><span class="p">(</span><span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">)</span>
<span class="c1"># using the outer self argument.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">trigger_fn</span><span class="o">.</span><span class="n">on_merge</span><span class="p">(</span>
<span class="n">to_be_merged</span><span class="p">,</span> <span class="n">merge_result</span><span class="p">,</span> <span class="n">state</span><span class="o">.</span><span class="n">at</span><span class="p">(</span><span class="n">merge_result</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">clock</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">window_fn</span><span class="o">.</span><span class="n">merge</span><span class="p">(</span><span class="n">TriggerMergeContext</span><span class="p">(</span><span class="n">all_windows</span><span class="p">))</span>
<span class="n">merged_windows_to_elements</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">)</span>
<span class="k">for</span> <span class="n">window</span><span class="p">,</span> <span class="n">values</span> <span class="ow">in</span> <span class="n">windows_to_elements</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">while</span> <span class="n">window</span> <span class="ow">in</span> <span class="n">merged_away</span><span class="p">:</span>
<span class="n">window</span> <span class="o">=</span> <span class="n">merged_away</span><span class="p">[</span><span class="n">window</span><span class="p">]</span>
<span class="n">merged_windows_to_elements</span><span class="p">[</span><span class="n">window</span><span class="p">]</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">values</span><span class="p">)</span>
<span class="n">windows_to_elements</span> <span class="o">=</span> <span class="n">merged_windows_to_elements</span>
<span class="k">for</span> <span class="n">window</span> <span class="ow">in</span> <span class="n">merged_away</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_HOLD</span><span class="p">)</span>
<span class="c1"># Next handle element adding.</span>
<span class="k">for</span> <span class="n">window</span><span class="p">,</span> <span class="n">elements</span> <span class="ow">in</span> <span class="n">windows_to_elements</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">if</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">TOMBSTONE</span><span class="p">):</span>
<span class="k">continue</span>
<span class="c1"># Add watermark hold.</span>
<span class="c1"># TODO(ccy): Add late data and garbage-collection hold support.</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_impl</span><span class="o">.</span><span class="n">merge</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span>
<span class="p">(</span>
<span class="n">element_output_time</span> <span class="k">for</span> <span class="n">element_output_time</span> <span class="ow">in</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_combiner_impl</span><span class="o">.</span><span class="n">assign_output_time</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span> <span class="k">for</span> <span class="n">unused_value</span><span class="p">,</span>
<span class="n">timestamp</span> <span class="ow">in</span> <span class="n">elements</span><span class="p">)</span>
<span class="k">if</span> <span class="n">element_output_time</span> <span class="o">&gt;=</span> <span class="n">output_watermark</span><span class="p">))</span>
<span class="k">if</span> <span class="n">output_time</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</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_HOLD</span><span class="p">,</span> <span class="n">output_time</span><span class="p">)</span>
<span class="n">context</span> <span class="o">=</span> <span class="n">state</span><span class="o">.</span><span class="n">at</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">clock</span><span class="p">)</span>
<span class="k">for</span> <span class="n">value</span><span class="p">,</span> <span class="n">unused_timestamp</span> <span class="ow">in</span> <span class="n">elements</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">ELEMENTS</span><span class="p">,</span> <span class="n">value</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">trigger_fn</span><span class="o">.</span><span class="n">on_element</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span>
<span class="c1"># Maybe fire this window.</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">trigger_fn</span><span class="o">.</span><span class="n">should_fire</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">input_watermark</span><span class="p">,</span>
<span class="n">window</span><span class="p">,</span>
<span class="n">context</span><span class="p">):</span>
<span class="n">finished</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">trigger_fn</span><span class="o">.</span><span class="n">on_fire</span><span class="p">(</span><span class="n">input_watermark</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span>
<span class="k">yield</span> <span class="bp">self</span><span class="o">.</span><span class="n">_output</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">finished</span><span class="p">,</span> <span class="n">state</span><span class="p">,</span> <span class="n">output_watermark</span><span class="p">,</span> <span class="kc">False</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">process_timer</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">window_id</span><span class="p">,</span>
<span class="n">unused_name</span><span class="p">,</span>
<span class="n">time_domain</span><span class="p">,</span>
<span class="n">timestamp</span><span class="p">,</span>
<span class="n">state</span><span class="p">,</span>
<span class="n">input_watermark</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">if</span> <span class="n">input_watermark</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">input_watermark</span> <span class="o">=</span> <span class="n">timestamp</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_merging</span><span class="p">:</span>
<span class="n">state</span> <span class="o">=</span> <span class="n">MergeableStateAdapter</span><span class="p">(</span><span class="n">state</span><span class="p">)</span>
<span class="n">window</span> <span class="o">=</span> <span class="n">state</span><span class="o">.</span><span class="n">get_window</span><span class="p">(</span><span class="n">window_id</span><span class="p">)</span>
<span class="k">if</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">TOMBSTONE</span><span class="p">):</span>
<span class="k">return</span>
<span class="k">if</span> <span class="n">time_domain</span> <span class="ow">in</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">TimeDomain</span><span class="o">.</span><span class="n">REAL_TIME</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">is_merging</span> <span class="ow">or</span> <span class="n">window</span> <span class="ow">in</span> <span class="n">state</span><span class="o">.</span><span class="n">known_windows</span><span class="p">():</span>
<span class="n">context</span> <span class="o">=</span> <span class="n">state</span><span class="o">.</span><span class="n">at</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">clock</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">trigger_fn</span><span class="o">.</span><span class="n">should_fire</span><span class="p">(</span><span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="n">finished</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">trigger_fn</span><span class="o">.</span><span class="n">on_fire</span><span class="p">(</span><span class="n">timestamp</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span>
<span class="k">yield</span> <span class="bp">self</span><span class="o">.</span><span class="n">_output</span><span class="p">(</span>
<span class="n">window</span><span class="p">,</span>
<span class="n">finished</span><span class="p">,</span>
<span class="n">state</span><span class="p">,</span>
<span class="n">timestamp</span><span class="p">,</span>
<span class="n">time_domain</span> <span class="o">==</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">WATERMARK</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">Exception</span><span class="p">(</span><span class="s1">&#39;Unexpected time domain: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">time_domain</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">finished</span><span class="p">,</span> <span class="n">state</span><span class="p">,</span> <span class="n">output_watermark</span><span class="p">,</span> <span class="n">maybe_ontime</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Output window and clean up if appropriate.&quot;&quot;&quot;</span>
<span class="n">index</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">INDEX</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">INDEX</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span>
<span class="k">if</span> <span class="n">output_watermark</span> <span class="o">&lt;=</span> <span class="n">window</span><span class="o">.</span><span class="n">max_timestamp</span><span class="p">():</span>
<span class="n">nonspeculative_index</span> <span class="o">=</span> <span class="o">-</span><span class="mi">1</span>
<span class="n">timing</span> <span class="o">=</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">PaneInfoTiming</span><span class="o">.</span><span class="n">EARLY</span>
<span class="k">if</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">NONSPECULATIVE_INDEX</span><span class="p">):</span>
<span class="n">nonspeculative_index</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">NONSPECULATIVE_INDEX</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">NONSPECULATIVE_INDEX</span><span class="p">,</span> <span class="mi">1</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;Watermark moved backwards in time &#39;</span>
<span class="s1">&#39;or late data moved window end forward.&#39;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">nonspeculative_index</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">NONSPECULATIVE_INDEX</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">NONSPECULATIVE_INDEX</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span>
<span class="n">timing</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">windowed_value</span><span class="o">.</span><span class="n">PaneInfoTiming</span><span class="o">.</span><span class="n">ON_TIME</span> <span class="k">if</span> <span class="n">maybe_ontime</span> <span class="ow">and</span>
<span class="n">nonspeculative_index</span> <span class="o">==</span> <span class="mi">0</span> <span class="k">else</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">PaneInfoTiming</span><span class="o">.</span><span class="n">LATE</span><span class="p">)</span>
<span class="n">pane_info</span> <span class="o">=</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">PaneInfo</span><span class="p">(</span>
<span class="n">index</span> <span class="o">==</span> <span class="mi">0</span><span class="p">,</span> <span class="n">finished</span><span class="p">,</span> <span class="n">timing</span><span class="p">,</span> <span class="n">index</span><span class="p">,</span> <span class="n">nonspeculative_index</span><span class="p">)</span>
<span class="n">values</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">ELEMENTS</span><span class="p">)</span>
<span class="k">if</span> <span class="n">finished</span><span class="p">:</span>
<span class="c1"># TODO(robertwb): allowed lateness</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">ELEMENTS</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">TOMBSTONE</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span>
<span class="k">elif</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="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">ELEMENTS</span><span class="p">)</span>
<span class="n">timestamp</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_HOLD</span><span class="p">)</span>
<span class="k">if</span> <span class="n">timestamp</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="c1"># If no watermark hold was set, output at end of window.</span>
<span class="n">timestamp</span> <span class="o">=</span> <span class="n">window</span><span class="o">.</span><span class="n">max_timestamp</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">output_watermark</span> <span class="o">&lt;</span> <span class="n">window</span><span class="o">.</span><span class="n">end</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">trigger_fn</span><span class="o">.</span><span class="n">has_ontime_pane</span><span class="p">():</span>
<span class="c1"># Hold the watermark in case there is an empty pane that needs to be fired</span>
<span class="c1"># at the end of the window.</span>
<span class="k">pass</span>
<span class="k">else</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_HOLD</span><span class="p">)</span>
<span class="k">return</span> <span class="n">WindowedValue</span><span class="p">(</span><span class="n">values</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="p">),</span> <span class="n">pane_info</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">InMemoryUnmergedState</span><span class="p">(</span><span class="n">UnmergedState</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;In-memory implementation of UnmergedState.</span>
<span class="sd"> Used for batch and testing.</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">defensive_copy</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="c1"># TODO(robertwb): Clean defensive_copy. It is too expensive in production.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timers</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">dict</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="k">lambda</span><span class="p">:</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">global_state</span> <span class="o">=</span> <span class="p">{}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">defensive_copy</span> <span class="o">=</span> <span class="n">defensive_copy</span>
<span class="k">def</span> <span class="nf">copy</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">cloned_object</span> <span class="o">=</span> <span class="n">InMemoryUnmergedState</span><span class="p">(</span><span class="n">defensive_copy</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">defensive_copy</span><span class="p">)</span>
<span class="n">cloned_object</span><span class="o">.</span><span class="n">timers</span> <span class="o">=</span> <span class="n">copy</span><span class="o">.</span><span class="n">deepcopy</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">timers</span><span class="p">)</span>
<span class="n">cloned_object</span><span class="o">.</span><span class="n">global_state</span> <span class="o">=</span> <span class="n">copy</span><span class="o">.</span><span class="n">deepcopy</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">global_state</span><span class="p">)</span>
<span class="k">for</span> <span class="n">window</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">:</span>
<span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">]:</span>
<span class="n">cloned_object</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="n">tag</span><span class="p">]</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="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="n">tag</span><span class="p">])</span>
<span class="k">return</span> <span class="n">cloned_object</span>
<span class="k">def</span> <span class="nf">set_global_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">defensive_copy</span><span class="p">:</span>
<span class="n">value</span> <span class="o">=</span> <span class="n">copy</span><span class="o">.</span><span class="n">deepcopy</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">global_state</span><span class="p">[</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span>
<span class="k">def</span> <span class="nf">get_global_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">default</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">global_state</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">,</span> <span class="n">default</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">set_timer</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timers</span><span class="p">[</span><span class="n">window</span><span class="p">][(</span><span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="p">)]</span> <span class="o">=</span> <span class="n">timestamp</span>
<span class="k">def</span> <span class="nf">clear_timer</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timers</span><span class="p">[</span><span class="n">window</span><span class="p">]</span><span class="o">.</span><span class="n">pop</span><span class="p">((</span><span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="p">),</span> <span class="kc">None</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">timers</span><span class="p">[</span><span class="n">window</span><span class="p">]:</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">timers</span><span class="p">[</span><span class="n">window</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">get_window</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window_id</span><span class="p">):</span>
<span class="k">return</span> <span class="n">window_id</span>
<span class="k">def</span> <span class="nf">add_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">defensive_copy</span><span class="p">:</span>
<span class="n">value</span> <span class="o">=</span> <span class="n">copy</span><span class="o">.</span><span class="n">deepcopy</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_CombiningValueStateTag</span><span class="p">):</span>
<span class="c1"># TODO(robertwb): Store merged accumulators.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_ListStateTag</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_SetStateTag</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_WatermarkHoldStateTag</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">]</span><span class="o">.</span><span class="n">append</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">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Invalid tag.&#39;</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="n">values</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">][</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_ReadModifyWriteStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="n">values</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_CombiningValueStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="n">tag</span><span class="o">.</span><span class="n">combine_fn</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">values</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_ListStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="n">values</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_SetStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="n">values</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">_WatermarkHoldStateTag</span><span class="p">):</span>
<span class="k">return</span> <span class="n">tag</span><span class="o">.</span><span class="n">timestamp_combiner_impl</span><span class="o">.</span><span class="n">combine_all</span><span class="p">(</span><span class="n">values</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;Invalid tag.&#39;</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">clear_state</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">[</span><span class="n">window</span><span class="p">]</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">tag</span><span class="o">.</span><span class="n">tag</span><span class="p">,</span> <span class="kc">None</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">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">state</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_timers</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">clear</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">watermark</span><span class="o">=</span><span class="n">MAX_TIMESTAMP</span><span class="p">,</span> <span class="n">processing_time</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets expired timers and reports if there</span>
<span class="sd"> are any realtime timers set per state.</span>
<span class="sd"> Expiration is measured against the watermark for event-time timers,</span>
<span class="sd"> and against a wall clock for processing-time timers.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">expired</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">has_realtime_timer</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">for</span> <span class="n">window</span><span class="p">,</span> <span class="n">timers</span> <span class="ow">in</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">timers</span><span class="o">.</span><span class="n">items</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">time_domain</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="p">),</span> <span class="n">timestamp</span> <span class="ow">in</span> <span class="nb">list</span><span class="p">(</span>
<span class="n">timers</span><span class="o">.</span><span class="n">items</span><span class="p">()):</span>
<span class="k">if</span> <span class="n">time_domain</span> <span class="o">==</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">REAL_TIME</span><span class="p">:</span>
<span class="n">time_marker</span> <span class="o">=</span> <span class="n">processing_time</span>
<span class="n">has_realtime_timer</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">elif</span> <span class="n">time_domain</span> <span class="o">==</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">WATERMARK</span><span class="p">:</span>
<span class="n">time_marker</span> <span class="o">=</span> <span class="n">watermark</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">error</span><span class="p">(</span>
<span class="s1">&#39;TimeDomain error: No timers defined for time domain </span><span class="si">%s</span><span class="s1">.&#39;</span><span class="p">,</span>
<span class="n">time_domain</span><span class="p">)</span>
<span class="k">if</span> <span class="n">timestamp</span> <span class="o">&lt;=</span> <span class="n">time_marker</span><span class="p">:</span>
<span class="n">expired</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="p">)))</span>
<span class="k">if</span> <span class="n">clear</span><span class="p">:</span>
<span class="k">del</span> <span class="n">timers</span><span class="p">[(</span><span class="n">name</span><span class="p">,</span> <span class="n">time_domain</span><span class="p">,</span> <span class="n">dynamic_timer_tag</span><span class="p">)]</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">timers</span> <span class="ow">and</span> <span class="n">clear</span><span class="p">:</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">timers</span><span class="p">[</span><span class="n">window</span><span class="p">]</span>
<span class="k">return</span> <span class="n">expired</span><span class="p">,</span> <span class="n">has_realtime_timer</span>
<span class="k">def</span> <span class="nf">get_and_clear_timers</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermark</span><span class="o">=</span><span class="n">MAX_TIMESTAMP</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_timers</span><span class="p">(</span><span class="n">clear</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">watermark</span><span class="o">=</span><span class="n">watermark</span><span class="p">)[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">get_earliest_hold</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">earliest_hold</span> <span class="o">=</span> <span class="n">MAX_TIMESTAMP</span>
<span class="k">for</span> <span class="n">unused_window</span><span class="p">,</span> <span class="n">tagged_states</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/18441): currently, this</span>
<span class="c1"># assumes that the watermark hold tag is named &quot;watermark&quot;. This is</span>
<span class="c1"># currently only true because the only place watermark holds are set is</span>
<span class="c1"># in the GeneralTriggerDriver, where we use this name. We should fix</span>
<span class="c1"># this by allowing enumeration of the tag types used in adding state.</span>
<span class="k">if</span> <span class="s1">&#39;watermark&#39;</span> <span class="ow">in</span> <span class="n">tagged_states</span> <span class="ow">and</span> <span class="n">tagged_states</span><span class="p">[</span><span class="s1">&#39;watermark&#39;</span><span class="p">]:</span>
<span class="n">hold</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">tagged_states</span><span class="p">[</span><span class="s1">&#39;watermark&#39;</span><span class="p">])</span> <span class="o">-</span> <span class="n">TIME_GRANULARITY</span>
<span class="n">earliest_hold</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">earliest_hold</span><span class="p">,</span> <span class="n">hold</span><span class="p">)</span>
<span class="k">return</span> <span class="n">earliest_hold</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="n">state_str</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</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="n">key</span><span class="p">,</span> <span class="nb">dict</span><span class="p">(</span><span class="n">state</span><span class="p">))</span> <span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">state</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="o">.</span><span class="n">items</span><span class="p">())</span>
<span class="k">return</span> <span class="s1">&#39;timers: </span><span class="si">%s</span><span class="se">\n</span><span class="s1">state: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="nb">dict</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">timers</span><span class="p">),</span> <span class="n">state_str</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>