blob: e144652808ade6dd80f5842b3c34ace2b4f13148 [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.runners.direct.evaluation_context &mdash; Apache Beam documentation</title>
<link rel="stylesheet" href="../../../../_static/css/theme.css" type="text/css" />
<link rel="index" title="Index"
href="../../../../genindex.html"/>
<link rel="search" title="Search" href="../../../../search.html"/>
<link rel="top" title="Apache Beam documentation" href="../../../../index.html"/>
<link rel="up" title="Module code" href="../../../index.html"/>
<script src="../../../../_static/js/modernizr.min.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search">
<a href="../../../../index.html" class="icon icon-home"> Apache Beam
</a>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.internal.html">apache_beam.internal package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.tools.html">apache_beam.tools package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.utils.html">apache_beam.utils package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.version.html">apache_beam.version module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../../index.html">Docs</a> &raquo;</li>
<li><a href="../../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.runners.direct.evaluation_context</li>
<li class="wy-breadcrumbs-aside">
</li>
</ul>
<hr/>
</div>
<div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article">
<div itemprop="articleBody">
<h1>Source code for apache_beam.runners.direct.evaluation_context</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;EvaluationContext tracks global state, triggers and watermarks.&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span>
<span class="kn">import</span> <span class="nn">collections</span>
<span class="kn">import</span> <span class="nn">threading</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">object</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.direct_metrics</span> <span class="k">import</span> <span class="n">DirectMetrics</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.executor</span> <span class="k">import</span> <span class="n">TransformExecutor</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.watermark_manager</span> <span class="k">import</span> <span class="n">WatermarkManager</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="k">import</span> <span class="n">sideinputs</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="k">import</span> <span class="n">InMemoryUnmergedState</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="k">import</span> <span class="n">counters</span>
<span class="k">class</span> <span class="nc">_ExecutionContext</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watermarks</span><span class="p">,</span> <span class="n">keyed_states</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">watermarks</span> <span class="o">=</span> <span class="n">watermarks</span>
<span class="bp">self</span><span class="o">.</span><span class="n">keyed_states</span> <span class="o">=</span> <span class="n">keyed_states</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_step_context</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">get_step_context</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">_step_context</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_step_context</span> <span class="o">=</span> <span class="n">DirectStepContext</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">keyed_states</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_step_context</span>
<span class="k">def</span> <span class="nf">reset</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_step_context</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">class</span> <span class="nc">_SideInputView</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">view</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_view</span> <span class="o">=</span> <span class="n">view</span>
<span class="bp">self</span><span class="o">.</span><span class="n">blocked_tasks</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">deque</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">elements</span> <span class="o">=</span> <span class="p">[]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">value</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">watermark</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">elements_string</span> <span class="o">=</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">elm</span><span class="p">)</span> <span class="k">for</span> <span class="n">elm</span> <span class="ow">in</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="bp">self</span><span class="o">.</span><span class="n">elements</span> <span class="k">else</span> <span class="s1">&#39;[]&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="s1">&#39;_SideInputView(elements=</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">elements_string</span>
<span class="k">class</span> <span class="nc">_SideInputsContainer</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;An in-process container for side inputs.</span>
<span class="sd"> It provides methods for blocking until a side-input is available and writing</span>
<span class="sd"> to a side input.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">side_inputs</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_lock</span> <span class="o">=</span> <span class="n">threading</span><span class="o">.</span><span class="n">Lock</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_views</span> <span class="o">=</span> <span class="p">{}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_to_side_inputs</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="bp">self</span><span class="o">.</span><span class="n">_side_input_to_blocked_tasks</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">side</span> <span class="ow">in</span> <span class="n">side_inputs</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_views</span><span class="p">[</span><span class="n">side</span><span class="p">]</span> <span class="o">=</span> <span class="n">_SideInputView</span><span class="p">(</span><span class="n">side</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_to_side_inputs</span><span class="p">[</span><span class="n">side</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">producer</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">side</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">views_string</span> <span class="o">=</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">elm</span><span class="p">)</span> <span class="k">for</span> <span class="n">elm</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_views</span><span class="o">.</span><span class="n">values</span><span class="p">())</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_views</span> <span class="k">else</span> <span class="s1">&#39;[]&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="s1">&#39;_SideInputsContainer(_views=</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="n">views_string</span>
<span class="k">def</span> <span class="nf">get_value_or_block_until_ready</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">side_input</span><span class="p">,</span> <span class="n">task</span><span class="p">,</span> <span class="n">block_until</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns the value of a view whose task is unblocked or blocks its task.</span>
<span class="sd"> It gets the value of a view whose watermark has been updated and</span>
<span class="sd"> surpasses a given value.</span>
<span class="sd"> Args:</span>
<span class="sd"> side_input: ``_UnpickledSideInput`` value.</span>
<span class="sd"> task: ``TransformExecutor`` task waiting on a side input.</span>
<span class="sd"> block_until: Timestamp after which the task gets unblocked.</span>
<span class="sd"> Returns:</span>
<span class="sd"> The ``SideInputMap`` value of a view when the tasks it blocks are</span>
<span class="sd"> unblocked. Otherwise, None.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_lock</span><span class="p">:</span>
<span class="n">view</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_views</span><span class="p">[</span><span class="n">side_input</span><span class="p">]</span>
<span class="k">if</span> <span class="n">view</span><span class="o">.</span><span class="n">watermark</span> <span class="ow">and</span> <span class="n">view</span><span class="o">.</span><span class="n">watermark</span><span class="o">.</span><span class="n">output_watermark</span> <span class="o">&gt;=</span> <span class="n">block_until</span><span class="p">:</span>
<span class="n">view</span><span class="o">.</span><span class="n">value</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pvalue_to_value</span><span class="p">(</span><span class="n">side_input</span><span class="p">,</span> <span class="n">view</span><span class="o">.</span><span class="n">elements</span><span class="p">)</span>
<span class="k">return</span> <span class="n">view</span><span class="o">.</span><span class="n">value</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">view</span><span class="o">.</span><span class="n">blocked_tasks</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">task</span><span class="p">,</span> <span class="n">block_until</span><span class="p">))</span>
<span class="n">task</span><span class="o">.</span><span class="n">blocked</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">def</span> <span class="nf">add_values</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">side_input</span><span class="p">,</span> <span class="n">values</span><span class="p">):</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_lock</span><span class="p">:</span>
<span class="n">view</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_views</span><span class="p">[</span><span class="n">side_input</span><span class="p">]</span>
<span class="n">view</span><span class="o">.</span><span class="n">elements</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="k">def</span> <span class="nf">update_watermarks_for_transform_and_unblock_tasks</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span>
<span class="n">ptransform</span><span class="p">,</span>
<span class="n">watermark</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Updates _SideInputsContainer after a watermark update and unbloks tasks.</span>
<span class="sd"> It traverses the list of side inputs per PTransform and calls</span>
<span class="sd"> _update_watermarks_for_side_input_and_unblock_tasks to unblock tasks.</span>
<span class="sd"> Args:</span>
<span class="sd"> ptransform: Value of a PTransform.</span>
<span class="sd"> watermark: Value of the watermark after an update for a PTransform.</span>
<span class="sd"> Returns:</span>
<span class="sd"> Tasks that get unblocked as a result of the watermark advancing.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">unblocked_tasks</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">side</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_transform_to_side_inputs</span><span class="p">[</span><span class="n">ptransform</span><span class="p">]:</span>
<span class="n">unblocked_tasks</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">_update_watermarks_for_side_input_and_unblock_tasks</span><span class="p">(</span>
<span class="n">side</span><span class="p">,</span> <span class="n">watermark</span><span class="p">))</span>
<span class="k">return</span> <span class="n">unblocked_tasks</span>
<span class="k">def</span> <span class="nf">_update_watermarks_for_side_input_and_unblock_tasks</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span>
<span class="n">side_input</span><span class="p">,</span>
<span class="n">watermark</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Helps update _SideInputsContainer after a watermark update.</span>
<span class="sd"> For each view of the side input, it updates the value of the watermark</span>
<span class="sd"> recorded when the watermark moved and unblocks tasks accordingly.</span>
<span class="sd"> Args:</span>
<span class="sd"> side_input: ``_UnpickledSideInput`` value.</span>
<span class="sd"> watermark: Value of the watermark after an update for a PTransform.</span>
<span class="sd"> Returns:</span>
<span class="sd"> Tasks that get unblocked as a result of the watermark advancing.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_lock</span><span class="p">:</span>
<span class="n">view</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_views</span><span class="p">[</span><span class="n">side_input</span><span class="p">]</span>
<span class="n">view</span><span class="o">.</span><span class="n">watermark</span> <span class="o">=</span> <span class="n">watermark</span>
<span class="n">unblocked_tasks</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">tasks_just_unblocked</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">task</span><span class="p">,</span> <span class="n">block_until</span> <span class="ow">in</span> <span class="n">view</span><span class="o">.</span><span class="n">blocked_tasks</span><span class="p">:</span>
<span class="k">if</span> <span class="n">watermark</span><span class="o">.</span><span class="n">output_watermark</span> <span class="o">&gt;=</span> <span class="n">block_until</span><span class="p">:</span>
<span class="n">view</span><span class="o">.</span><span class="n">value</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pvalue_to_value</span><span class="p">(</span><span class="n">side_input</span><span class="p">,</span> <span class="n">view</span><span class="o">.</span><span class="n">elements</span><span class="p">)</span>
<span class="n">unblocked_tasks</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">task</span><span class="p">)</span>
<span class="n">tasks_just_unblocked</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">task</span><span class="p">,</span> <span class="n">block_until</span><span class="p">))</span>
<span class="n">task</span><span class="o">.</span><span class="n">blocked</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">for</span> <span class="n">task</span> <span class="ow">in</span> <span class="n">tasks_just_unblocked</span><span class="p">:</span>
<span class="n">view</span><span class="o">.</span><span class="n">blocked_tasks</span><span class="o">.</span><span class="n">remove</span><span class="p">(</span><span class="n">task</span><span class="p">)</span>
<span class="k">return</span> <span class="n">unblocked_tasks</span>
<span class="k">def</span> <span class="nf">_pvalue_to_value</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">side_input</span><span class="p">,</span> <span class="n">values</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Given a side input, returns the associated value in its requested form.</span>
<span class="sd"> Args:</span>
<span class="sd"> side_input: _UnpickledSideInput object.</span>
<span class="sd"> values: Iterable values associated with the side input.</span>
<span class="sd"> Returns:</span>
<span class="sd"> The side input in its requested form.</span>
<span class="sd"> Raises:</span>
<span class="sd"> ValueError: If values cannot be converted into the requested form.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">sideinputs</span><span class="o">.</span><span class="n">SideInputMap</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">side_input</span><span class="p">),</span>
<span class="n">side_input</span><span class="o">.</span><span class="n">_view_options</span><span class="p">(),</span>
<span class="n">values</span><span class="p">)</span>
<div class="viewcode-block" id="EvaluationContext"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext">[docs]</a><span class="k">class</span> <span class="nc">EvaluationContext</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Evaluation context with the global state information of the pipeline.</span>
<span class="sd"> The evaluation context for a specific pipeline being executed by the</span>
<span class="sd"> DirectRunner. Contains state shared within the execution across all</span>
<span class="sd"> transforms.</span>
<span class="sd"> EvaluationContext contains shared state for an execution of the</span>
<span class="sd"> DirectRunner that can be used while evaluating a PTransform. This</span>
<span class="sd"> consists of views into underlying state and watermark implementations, access</span>
<span class="sd"> to read and write side inputs, and constructing counter sets and</span>
<span class="sd"> execution contexts. This includes executing callbacks asynchronously when</span>
<span class="sd"> state changes to the appropriate point (e.g. when a side input is</span>
<span class="sd"> requested and known to be empty).</span>
<span class="sd"> EvaluationContext also handles results by committing finalizing</span>
<span class="sd"> bundles based on the current global state and updating the global state</span>
<span class="sd"> appropriately. This includes updating the per-(step,key) state, updating</span>
<span class="sd"> global watermarks, and executing any callbacks that can be executed.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline_options</span><span class="p">,</span> <span class="n">bundle_factory</span><span class="p">,</span> <span class="n">root_transforms</span><span class="p">,</span>
<span class="n">value_to_consumers</span><span class="p">,</span> <span class="n">step_names</span><span class="p">,</span> <span class="n">views</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">pipeline_options</span> <span class="o">=</span> <span class="n">pipeline_options</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_bundle_factory</span> <span class="o">=</span> <span class="n">bundle_factory</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_root_transforms</span> <span class="o">=</span> <span class="n">root_transforms</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_value_to_consumers</span> <span class="o">=</span> <span class="n">value_to_consumers</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_step_names</span> <span class="o">=</span> <span class="n">step_names</span>
<span class="bp">self</span><span class="o">.</span><span class="n">views</span> <span class="o">=</span> <span class="n">views</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pcollection_to_views</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">view</span> <span class="ow">in</span> <span class="n">views</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pcollection_to_views</span><span class="p">[</span><span class="n">view</span><span class="o">.</span><span class="n">pvalue</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">view</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_keyed_states</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initialize_keyed_states</span><span class="p">(</span>
<span class="n">root_transforms</span><span class="p">,</span> <span class="n">value_to_consumers</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_side_inputs_container</span> <span class="o">=</span> <span class="n">_SideInputsContainer</span><span class="p">(</span><span class="n">views</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_watermark_manager</span> <span class="o">=</span> <span class="n">WatermarkManager</span><span class="p">(</span>
<span class="n">clock</span><span class="p">,</span> <span class="n">root_transforms</span><span class="p">,</span> <span class="n">value_to_consumers</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_keyed_states</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pending_unblocked_tasks</span> <span class="o">=</span> <span class="p">[]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_counter_factory</span> <span class="o">=</span> <span class="n">counters</span><span class="o">.</span><span class="n">CounterFactory</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_metrics</span> <span class="o">=</span> <span class="n">DirectMetrics</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_lock</span> <span class="o">=</span> <span class="n">threading</span><span class="o">.</span><span class="n">Lock</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_initialize_keyed_states</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">root_transforms</span><span class="p">,</span> <span class="n">value_to_consumers</span><span class="p">):</span>
<span class="n">transform_keyed_states</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">transform</span> <span class="ow">in</span> <span class="n">root_transforms</span><span class="p">:</span>
<span class="n">transform_keyed_states</span><span class="p">[</span><span class="n">transform</span><span class="p">]</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">consumers</span> <span class="ow">in</span> <span class="n">value_to_consumers</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="k">for</span> <span class="n">consumer</span> <span class="ow">in</span> <span class="n">consumers</span><span class="p">:</span>
<span class="n">transform_keyed_states</span><span class="p">[</span><span class="n">consumer</span><span class="p">]</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">return</span> <span class="n">transform_keyed_states</span>
<div class="viewcode-block" id="EvaluationContext.metrics"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.metrics">[docs]</a> <span class="k">def</span> <span class="nf">metrics</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># TODO. Should this be made a @property?</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_metrics</span></div>
<div class="viewcode-block" id="EvaluationContext.is_root_transform"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.is_root_transform">[docs]</a> <span class="k">def</span> <span class="nf">is_root_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">return</span> <span class="n">applied_ptransform</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_root_transforms</span></div>
<div class="viewcode-block" id="EvaluationContext.handle_result"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.handle_result">[docs]</a> <span class="k">def</span> <span class="nf">handle_result</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">completed_bundle</span><span class="p">,</span> <span class="n">completed_timers</span><span class="p">,</span> <span class="n">result</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Handle the provided result produced after evaluating the input bundle.</span>
<span class="sd"> Handle the provided TransformResult, produced after evaluating</span>
<span class="sd"> the provided committed bundle (potentially None, if the result of a root</span>
<span class="sd"> PTransform).</span>
<span class="sd"> The result is the output of running the transform contained in the</span>
<span class="sd"> TransformResult on the contents of the provided bundle.</span>
<span class="sd"> Args:</span>
<span class="sd"> completed_bundle: the bundle that was processed to produce the result.</span>
<span class="sd"> completed_timers: the timers that were delivered to produce the</span>
<span class="sd"> completed_bundle.</span>
<span class="sd"> result: the TransformResult of evaluating the input bundle</span>
<span class="sd"> Returns:</span>
<span class="sd"> the committed bundles contained within the handled result.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_lock</span><span class="p">:</span>
<span class="n">committed_bundles</span><span class="p">,</span> <span class="n">unprocessed_bundles</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_commit_bundles</span><span class="p">(</span>
<span class="n">result</span><span class="o">.</span><span class="n">uncommitted_output_bundles</span><span class="p">,</span>
<span class="n">result</span><span class="o">.</span><span class="n">unprocessed_bundles</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_metrics</span><span class="o">.</span><span class="n">commit_logical</span><span class="p">(</span><span class="n">completed_bundle</span><span class="p">,</span>
<span class="n">result</span><span class="o">.</span><span class="n">logical_metric_updates</span><span class="p">)</span>
<span class="c1"># If the result is for a view, update side inputs container.</span>
<span class="k">if</span> <span class="p">(</span><span class="n">result</span><span class="o">.</span><span class="n">uncommitted_output_bundles</span>
<span class="ow">and</span> <span class="n">result</span><span class="o">.</span><span class="n">uncommitted_output_bundles</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">pcollection</span>
<span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcollection_to_views</span><span class="p">):</span>
<span class="k">for</span> <span class="n">view</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcollection_to_views</span><span class="p">[</span>
<span class="n">result</span><span class="o">.</span><span class="n">uncommitted_output_bundles</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">pcollection</span><span class="p">]:</span>
<span class="k">for</span> <span class="n">committed_bundle</span> <span class="ow">in</span> <span class="n">committed_bundles</span><span class="p">:</span>
<span class="c1"># side_input must be materialized.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_side_inputs_container</span><span class="o">.</span><span class="n">add_values</span><span class="p">(</span>
<span class="n">view</span><span class="p">,</span>
<span class="n">committed_bundle</span><span class="o">.</span><span class="n">get_elements_iterable</span><span class="p">(</span><span class="n">make_copy</span><span class="o">=</span><span class="kc">True</span><span class="p">))</span>
<span class="c1"># Tasks generated from unblocked side inputs as the watermark progresses.</span>
<span class="n">tasks</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_watermark_manager</span><span class="o">.</span><span class="n">update_watermarks</span><span class="p">(</span>
<span class="n">completed_bundle</span><span class="p">,</span> <span class="n">result</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="n">completed_timers</span><span class="p">,</span>
<span class="n">committed_bundles</span><span class="p">,</span> <span class="n">unprocessed_bundles</span><span class="p">,</span> <span class="n">result</span><span class="o">.</span><span class="n">keyed_watermark_holds</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_side_inputs_container</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pending_unblocked_tasks</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">tasks</span><span class="p">)</span>
<span class="k">if</span> <span class="n">result</span><span class="o">.</span><span class="n">counters</span><span class="p">:</span>
<span class="k">for</span> <span class="n">counter</span> <span class="ow">in</span> <span class="n">result</span><span class="o">.</span><span class="n">counters</span><span class="p">:</span>
<span class="n">merged_counter</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_counter_factory</span><span class="o">.</span><span class="n">get_counter</span><span class="p">(</span>
<span class="n">counter</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> <span class="n">counter</span><span class="o">.</span><span class="n">combine_fn</span><span class="p">)</span>
<span class="n">merged_counter</span><span class="o">.</span><span class="n">accumulator</span><span class="o">.</span><span class="n">merge</span><span class="p">([</span><span class="n">counter</span><span class="o">.</span><span class="n">accumulator</span><span class="p">])</span>
<span class="c1"># Commit partial GBK states</span>
<span class="n">existing_keyed_state</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_transform_keyed_states</span><span class="p">[</span><span class="n">result</span><span class="o">.</span><span class="n">transform</span><span class="p">]</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">result</span><span class="o">.</span><span class="n">partial_keyed_state</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">existing_keyed_state</span><span class="p">[</span><span class="n">k</span><span class="p">]</span> <span class="o">=</span> <span class="n">v</span>
<span class="k">return</span> <span class="n">committed_bundles</span></div>
<div class="viewcode-block" id="EvaluationContext.get_aggregator_values"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.get_aggregator_values">[docs]</a> <span class="k">def</span> <span class="nf">get_aggregator_values</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">aggregator_or_name</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_counter_factory</span><span class="o">.</span><span class="n">get_aggregator_values</span><span class="p">(</span><span class="n">aggregator_or_name</span><span class="p">)</span></div>
<div class="viewcode-block" id="EvaluationContext.schedule_pending_unblocked_tasks"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.schedule_pending_unblocked_tasks">[docs]</a> <span class="k">def</span> <span class="nf">schedule_pending_unblocked_tasks</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">executor_service</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pending_unblocked_tasks</span><span class="p">:</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_lock</span><span class="p">:</span>
<span class="k">for</span> <span class="n">task</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pending_unblocked_tasks</span><span class="p">:</span>
<span class="n">executor_service</span><span class="o">.</span><span class="n">submit</span><span class="p">(</span><span class="n">task</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pending_unblocked_tasks</span> <span class="o">=</span> <span class="p">[]</span></div>
<span class="k">def</span> <span class="nf">_commit_bundles</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">uncommitted_bundles</span><span class="p">,</span> <span class="n">unprocessed_bundles</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Commits bundles and returns a immutable set of committed bundles.&quot;&quot;&quot;</span>
<span class="k">for</span> <span class="n">in_progress_bundle</span> <span class="ow">in</span> <span class="n">uncommitted_bundles</span><span class="p">:</span>
<span class="n">producing_applied_ptransform</span> <span class="o">=</span> <span class="n">in_progress_bundle</span><span class="o">.</span><span class="n">pcollection</span><span class="o">.</span><span class="n">producer</span>
<span class="n">watermarks</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_watermark_manager</span><span class="o">.</span><span class="n">get_watermarks</span><span class="p">(</span>
<span class="n">producing_applied_ptransform</span><span class="p">)</span>
<span class="n">in_progress_bundle</span><span class="o">.</span><span class="n">commit</span><span class="p">(</span><span class="n">watermarks</span><span class="o">.</span><span class="n">synchronized_processing_output_time</span><span class="p">)</span>
<span class="k">for</span> <span class="n">unprocessed_bundle</span> <span class="ow">in</span> <span class="n">unprocessed_bundles</span><span class="p">:</span>
<span class="n">unprocessed_bundle</span><span class="o">.</span><span class="n">commit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">uncommitted_bundles</span><span class="p">),</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">unprocessed_bundles</span><span class="p">)</span>
<div class="viewcode-block" id="EvaluationContext.get_execution_context"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.get_execution_context">[docs]</a> <span class="k">def</span> <span class="nf">get_execution_context</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_ExecutionContext</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_watermark_manager</span><span class="o">.</span><span class="n">get_watermarks</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="p">),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_keyed_states</span><span class="p">[</span><span class="n">applied_ptransform</span><span class="p">])</span></div>
<div class="viewcode-block" id="EvaluationContext.create_bundle"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.create_bundle">[docs]</a> <span class="k">def</span> <span class="nf">create_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">output_pcollection</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Create an uncommitted bundle for the specified PCollection.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_bundle_factory</span><span class="o">.</span><span class="n">create_bundle</span><span class="p">(</span><span class="n">output_pcollection</span><span class="p">)</span></div>
<div class="viewcode-block" id="EvaluationContext.create_empty_committed_bundle"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.create_empty_committed_bundle">[docs]</a> <span class="k">def</span> <span class="nf">create_empty_committed_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">output_pcollection</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Create empty bundle useful for triggering evaluation.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_bundle_factory</span><span class="o">.</span><span class="n">create_empty_committed_bundle</span><span class="p">(</span>
<span class="n">output_pcollection</span><span class="p">)</span></div>
<div class="viewcode-block" id="EvaluationContext.extract_all_timers"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.extract_all_timers">[docs]</a> <span class="k">def</span> <span class="nf">extract_all_timers</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">_watermark_manager</span><span class="o">.</span><span class="n">extract_all_timers</span><span class="p">()</span></div>
<div class="viewcode-block" id="EvaluationContext.is_done"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.is_done">[docs]</a> <span class="k">def</span> <span class="nf">is_done</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Checks completion of a step or the pipeline.</span>
<span class="sd"> Args:</span>
<span class="sd"> transform: AppliedPTransform to check for completion.</span>
<span class="sd"> Returns:</span>
<span class="sd"> True if the step will not produce additional output. If transform is None</span>
<span class="sd"> returns true if all steps are done.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">transform</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_transform_done</span><span class="p">(</span><span class="n">transform</span><span class="p">)</span>
<span class="k">for</span> <span class="n">applied_ptransform</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_step_names</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_transform_done</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">return</span> <span class="kc">True</span></div>
<span class="k">def</span> <span class="nf">_is_transform_done</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">):</span>
<span class="n">tw</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_watermark_manager</span><span class="o">.</span><span class="n">get_watermarks</span><span class="p">(</span><span class="n">transform</span><span class="p">)</span>
<span class="k">return</span> <span class="n">tw</span><span class="o">.</span><span class="n">output_watermark</span> <span class="o">==</span> <span class="n">WatermarkManager</span><span class="o">.</span><span class="n">WATERMARK_POS_INF</span>
<div class="viewcode-block" id="EvaluationContext.get_value_or_block_until_ready"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.EvaluationContext.get_value_or_block_until_ready">[docs]</a> <span class="k">def</span> <span class="nf">get_value_or_block_until_ready</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">side_input</span><span class="p">,</span> <span class="n">task</span><span class="p">,</span> <span class="n">block_until</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">task</span><span class="p">,</span> <span class="n">TransformExecutor</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_side_inputs_container</span><span class="o">.</span><span class="n">get_value_or_block_until_ready</span><span class="p">(</span>
<span class="n">side_input</span><span class="p">,</span> <span class="n">task</span><span class="p">,</span> <span class="n">block_until</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="DirectUnmergedState"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.DirectUnmergedState">[docs]</a><span class="k">class</span> <span class="nc">DirectUnmergedState</span><span class="p">(</span><span class="n">InMemoryUnmergedState</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;UnmergedState implementation for the DirectRunner.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">DirectUnmergedState</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="n">defensive_copy</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span></div>
<div class="viewcode-block" id="DirectStepContext"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.DirectStepContext">[docs]</a><span class="k">class</span> <span class="nc">DirectStepContext</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Context for the currently-executing step.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">existing_keyed_state</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">existing_keyed_state</span> <span class="o">=</span> <span class="n">existing_keyed_state</span>
<span class="c1"># In order to avoid partial writes of a bundle, every time</span>
<span class="c1"># existing_keyed_state is accessed, a copy of the state is made</span>
<span class="c1"># to be transferred to the bundle state once the bundle is committed.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">partial_keyed_state</span> <span class="o">=</span> <span class="p">{}</span>
<div class="viewcode-block" id="DirectStepContext.get_keyed_state"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.evaluation_context.html#apache_beam.runners.direct.evaluation_context.DirectStepContext.get_keyed_state">[docs]</a> <span class="k">def</span> <span class="nf">get_keyed_state</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="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">existing_keyed_state</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">key</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">existing_keyed_state</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</span> <span class="n">DirectUnmergedState</span><span class="p">()</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">partial_keyed_state</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">key</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">partial_keyed_state</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">existing_keyed_state</span><span class="p">[</span><span class="n">key</span><span class="p">]</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">partial_keyed_state</span><span class="p">[</span><span class="n">key</span><span class="p">]</span></div></div>
</pre></div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright .
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
var DOCUMENTATION_OPTIONS = {
URL_ROOT:'../../../../',
VERSION:'',
COLLAPSE_INDEX:false,
FILE_SUFFIX:'.html',
HAS_SOURCE: true,
SOURCELINK_SUFFIX: '.txt'
};
</script>
<script type="text/javascript" src="../../../../_static/jquery.js"></script>
<script type="text/javascript" src="../../../../_static/underscore.js"></script>
<script type="text/javascript" src="../../../../_static/doctools.js"></script>
<script type="text/javascript" src="../../../../_static/js/theme.js"></script>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.StickyNav.enable();
});
</script>
</body>
</html>