blob: 24de7b8f1449158fbe1793ed0774b14a547df89e [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.executor &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.executor</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.executor</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;An executor that schedules and executes applied ptransforms.&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span>
<span class="kn">import</span> <span class="nn">collections</span>
<span class="kn">import</span> <span class="nn">itertools</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">import</span> <span class="nn">threading</span>
<span class="kn">import</span> <span class="nn">traceback</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">builtins</span> <span class="k">import</span> <span class="nb">range</span>
<span class="kn">from</span> <span class="nn">weakref</span> <span class="k">import</span> <span class="n">WeakValueDictionary</span>
<span class="kn">from</span> <span class="nn">future.moves</span> <span class="k">import</span> <span class="n">queue</span>
<span class="kn">from</span> <span class="nn">future.utils</span> <span class="k">import</span> <span class="n">raise_</span>
<span class="kn">from</span> <span class="nn">apache_beam.metrics.execution</span> <span class="k">import</span> <span class="n">MetricsContainer</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.worker</span> <span class="k">import</span> <span class="n">statesampler</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.utils</span> <span class="k">import</span> <span class="n">counters</span>
<span class="k">class</span> <span class="nc">_ExecutorService</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Thread pool for executing tasks in parallel.&quot;&quot;&quot;</span>
<span class="k">class</span> <span class="nc">CallableTask</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">call</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">state_sampler</span><span class="p">):</span>
<span class="k">pass</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">name</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">class</span> <span class="nc">_ExecutorServiceWorker</span><span class="p">(</span><span class="n">threading</span><span class="o">.</span><span class="n">Thread</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Worker thread for executing a single task at a time.&quot;&quot;&quot;</span>
<span class="c1"># Amount to block waiting for getting an item from the queue in seconds.</span>
<span class="n">TIMEOUT</span> <span class="o">=</span> <span class="mi">5</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">queue</span><span class="p">,</span> <span class="n">index</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">_ExecutorService</span><span class="o">.</span><span class="n">_ExecutorServiceWorker</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="bp">self</span><span class="o">.</span><span class="n">queue</span> <span class="o">=</span> <span class="n">queue</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_index</span> <span class="o">=</span> <span class="n">index</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_default_name</span> <span class="o">=</span> <span class="s1">&#39;ExecutorServiceWorker-&#39;</span> <span class="o">+</span> <span class="nb">str</span><span class="p">(</span><span class="n">index</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_name</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">shutdown_requested</span> <span class="o">=</span> <span class="kc">False</span>
<span class="c1"># Stop worker thread when main thread exits.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">daemon</span> <span class="o">=</span> <span class="kc">True</span>
<span class="bp">self</span><span class="o">.</span><span class="n">start</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_update_name</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">task</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">if</span> <span class="n">task</span> <span class="ow">and</span> <span class="n">task</span><span class="o">.</span><span class="n">name</span><span class="p">:</span>
<span class="n">name</span> <span class="o">=</span> <span class="n">task</span><span class="o">.</span><span class="n">name</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">name</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_default_name</span>
<span class="bp">self</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="s1">&#39;Thread: </span><span class="si">%d</span><span class="s1">, </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">_index</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="s1">&#39;executing&#39;</span> <span class="k">if</span> <span class="n">task</span> <span class="k">else</span> <span class="s1">&#39;idle&#39;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_get_task_or_none</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># Do not block indefinitely, otherwise we may not act for a requested</span>
<span class="c1"># shutdown.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">queue</span><span class="o">.</span><span class="n">get</span><span class="p">(</span>
<span class="n">timeout</span><span class="o">=</span><span class="n">_ExecutorService</span><span class="o">.</span><span class="n">_ExecutorServiceWorker</span><span class="o">.</span><span class="n">TIMEOUT</span><span class="p">)</span>
<span class="k">except</span> <span class="n">queue</span><span class="o">.</span><span class="n">Empty</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">run</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">state_sampler</span> <span class="o">=</span> <span class="n">statesampler</span><span class="o">.</span><span class="n">StateSampler</span><span class="p">(</span><span class="s1">&#39;&#39;</span><span class="p">,</span> <span class="n">counters</span><span class="o">.</span><span class="n">CounterFactory</span><span class="p">())</span>
<span class="n">statesampler</span><span class="o">.</span><span class="n">set_current_tracker</span><span class="p">(</span><span class="n">state_sampler</span><span class="p">)</span>
<span class="k">while</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">shutdown_requested</span><span class="p">:</span>
<span class="n">task</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_task_or_none</span><span class="p">()</span>
<span class="k">if</span> <span class="n">task</span><span class="p">:</span>
<span class="k">try</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">shutdown_requested</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_name</span><span class="p">(</span><span class="n">task</span><span class="p">)</span>
<span class="n">task</span><span class="o">.</span><span class="n">call</span><span class="p">(</span><span class="n">state_sampler</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_name</span><span class="p">()</span>
<span class="k">finally</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">queue</span><span class="o">.</span><span class="n">task_done</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">shutdown</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">shutdown_requested</span> <span class="o">=</span> <span class="kc">True</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">num_workers</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">queue</span> <span class="o">=</span> <span class="n">queue</span><span class="o">.</span><span class="n">Queue</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">workers</span> <span class="o">=</span> <span class="p">[</span><span class="n">_ExecutorService</span><span class="o">.</span><span class="n">_ExecutorServiceWorker</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">queue</span><span class="p">,</span> <span class="n">i</span><span class="p">)</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">num_workers</span><span class="p">)]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">shutdown_requested</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">def</span> <span class="nf">submit</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">task</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">_ExecutorService</span><span class="o">.</span><span class="n">CallableTask</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">shutdown_requested</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">queue</span><span class="o">.</span><span class="n">put</span><span class="p">(</span><span class="n">task</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">await_completion</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">for</span> <span class="n">worker</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">workers</span><span class="p">:</span>
<span class="n">worker</span><span class="o">.</span><span class="n">join</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">shutdown</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">shutdown_requested</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">for</span> <span class="n">worker</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">workers</span><span class="p">:</span>
<span class="n">worker</span><span class="o">.</span><span class="n">shutdown</span><span class="p">()</span>
<span class="c1"># Consume all the remaining items in the queue</span>
<span class="k">while</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">queue</span><span class="o">.</span><span class="n">empty</span><span class="p">():</span>
<span class="k">try</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">queue</span><span class="o">.</span><span class="n">get_nowait</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">queue</span><span class="o">.</span><span class="n">task_done</span><span class="p">()</span>
<span class="k">except</span> <span class="n">queue</span><span class="o">.</span><span class="n">Empty</span><span class="p">:</span>
<span class="k">continue</span>
<span class="c1"># All existing threads will eventually terminate (after they complete their</span>
<span class="c1"># last task).</span>
<span class="k">class</span> <span class="nc">_TransformEvaluationState</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">executor_service</span><span class="p">,</span> <span class="n">scheduled</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">executor_service</span> <span class="o">=</span> <span class="n">executor_service</span>
<span class="bp">self</span><span class="o">.</span><span class="n">scheduled</span> <span class="o">=</span> <span class="n">scheduled</span>
<span class="k">def</span> <span class="nf">schedule</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">work</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">scheduled</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">work</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">executor_service</span><span class="o">.</span><span class="n">submit</span><span class="p">(</span><span class="n">work</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">complete</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">completed_work</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">scheduled</span><span class="o">.</span><span class="n">remove</span><span class="p">(</span><span class="n">completed_work</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_ParallelEvaluationState</span><span class="p">(</span><span class="n">_TransformEvaluationState</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A TransformEvaluationState with unlimited parallelism.</span>
<span class="sd"> Any TransformExecutor scheduled will be immediately submitted to the</span>
<span class="sd"> ExecutorService.</span>
<span class="sd"> A principal use of this is for evaluators that can generate output bundles</span>
<span class="sd"> only using the input bundle (e.g. ParDo).</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">pass</span>
<span class="k">class</span> <span class="nc">_SerialEvaluationState</span><span class="p">(</span><span class="n">_TransformEvaluationState</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A TransformEvaluationState with a single work queue.</span>
<span class="sd"> Any TransformExecutor scheduled will be placed on the work queue. Only one</span>
<span class="sd"> item of work will be submitted to the ExecutorService at any time.</span>
<span class="sd"> A principal use of this is for evaluators that keeps a global state such as</span>
<span class="sd"> _GroupByKeyOnly.</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">executor_service</span><span class="p">,</span> <span class="n">scheduled</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">_SerialEvaluationState</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">executor_service</span><span class="p">,</span> <span class="n">scheduled</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">serial_queue</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">currently_evaluating</span> <span class="o">=</span> <span class="kc">None</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">complete</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">completed_work</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_currently_evaluating</span><span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="n">completed_work</span><span class="p">)</span>
<span class="nb">super</span><span class="p">(</span><span class="n">_SerialEvaluationState</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="n">complete</span><span class="p">(</span><span class="n">completed_work</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">schedule</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">new_work</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_currently_evaluating</span><span class="p">(</span><span class="n">new_work</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_update_currently_evaluating</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">new_work</span><span class="p">,</span> <span class="n">completed_work</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">if</span> <span class="n">new_work</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">serial_queue</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">new_work</span><span class="p">)</span>
<span class="k">if</span> <span class="n">completed_work</span><span class="p">:</span>
<span class="k">assert</span> <span class="bp">self</span><span class="o">.</span><span class="n">currently_evaluating</span> <span class="o">==</span> <span class="n">completed_work</span>
<span class="bp">self</span><span class="o">.</span><span class="n">currently_evaluating</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">serial_queue</span> <span class="ow">and</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">currently_evaluating</span><span class="p">:</span>
<span class="n">next_work</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">serial_queue</span><span class="o">.</span><span class="n">pop</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">currently_evaluating</span> <span class="o">=</span> <span class="n">next_work</span>
<span class="nb">super</span><span class="p">(</span><span class="n">_SerialEvaluationState</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="n">schedule</span><span class="p">(</span><span class="n">next_work</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_TransformExecutorServices</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Schedules and completes TransformExecutors.</span>
<span class="sd"> Controls the concurrency as appropriate for the applied transform the executor</span>
<span class="sd"> exists for.</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">executor_service</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor_service</span> <span class="o">=</span> <span class="n">executor_service</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_scheduled</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_parallel</span> <span class="o">=</span> <span class="n">_ParallelEvaluationState</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor_service</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_scheduled</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_serial_cache</span> <span class="o">=</span> <span class="n">WeakValueDictionary</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">parallel</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">_parallel</span>
<span class="k">def</span> <span class="nf">serial</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">step</span><span class="p">):</span>
<span class="n">cached</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_serial_cache</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">step</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">cached</span><span class="p">:</span>
<span class="n">cached</span> <span class="o">=</span> <span class="n">_SerialEvaluationState</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_executor_service</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_scheduled</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_serial_cache</span><span class="p">[</span><span class="n">step</span><span class="p">]</span> <span class="o">=</span> <span class="n">cached</span>
<span class="k">return</span> <span class="n">cached</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">executors</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">frozenset</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_scheduled</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_CompletionCallback</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;The default completion callback.</span>
<span class="sd"> The default completion callback is used to complete transform evaluations</span>
<span class="sd"> that are triggered due to the arrival of elements from an upstream transform,</span>
<span class="sd"> or for a source transform.</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">evaluation_context</span><span class="p">,</span> <span class="n">all_updates</span><span class="p">,</span> <span class="n">timer_firings</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_evaluation_context</span> <span class="o">=</span> <span class="n">evaluation_context</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_all_updates</span> <span class="o">=</span> <span class="n">all_updates</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_timer_firings</span> <span class="o">=</span> <span class="n">timer_firings</span> <span class="ow">or</span> <span class="p">[]</span>
<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">transform_executor</span><span class="p">,</span> <span class="n">input_committed_bundle</span><span class="p">,</span>
<span class="n">transform_result</span><span class="p">):</span>
<span class="n">output_committed_bundles</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_evaluation_context</span><span class="o">.</span><span class="n">handle_result</span><span class="p">(</span>
<span class="n">input_committed_bundle</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_timer_firings</span><span class="p">,</span> <span class="n">transform_result</span><span class="p">)</span>
<span class="k">for</span> <span class="n">output_committed_bundle</span> <span class="ow">in</span> <span class="n">output_committed_bundles</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_all_updates</span><span class="o">.</span><span class="n">offer</span><span class="p">(</span><span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_ExecutorUpdate</span><span class="p">(</span>
<span class="n">transform_executor</span><span class="p">,</span>
<span class="n">committed_bundle</span><span class="o">=</span><span class="n">output_committed_bundle</span><span class="p">))</span>
<span class="k">for</span> <span class="n">unprocessed_bundle</span> <span class="ow">in</span> <span class="n">transform_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">_all_updates</span><span class="o">.</span><span class="n">offer</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_ExecutorUpdate</span><span class="p">(</span>
<span class="n">transform_executor</span><span class="p">,</span>
<span class="n">unprocessed_bundle</span><span class="o">=</span><span class="n">unprocessed_bundle</span><span class="p">))</span>
<span class="k">return</span> <span class="n">output_committed_bundles</span>
<span class="k">def</span> <span class="nf">handle_exception</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_executor</span><span class="p">,</span> <span class="n">exception</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_all_updates</span><span class="o">.</span><span class="n">offer</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_ExecutorUpdate</span><span class="p">(</span>
<span class="n">transform_executor</span><span class="p">,</span> <span class="n">exception</span><span class="o">=</span><span class="n">exception</span><span class="p">))</span>
<div class="viewcode-block" id="TransformExecutor"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.executor.html#apache_beam.runners.direct.executor.TransformExecutor">[docs]</a><span class="k">class</span> <span class="nc">TransformExecutor</span><span class="p">(</span><span class="n">_ExecutorService</span><span class="o">.</span><span class="n">CallableTask</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> TransformExecutor will evaluate a bundle using an applied ptransform.</span>
<span class="sd"> A CallableTask responsible for constructing a TransformEvaluator and</span>
<span class="sd"> evaluating it on some bundle of input, and registering the result using the</span>
<span class="sd"> completion callback.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">_MAX_RETRY_PER_BUNDLE</span> <span class="o">=</span> <span class="mi">4</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">transform_evaluator_registry</span><span class="p">,</span> <span class="n">evaluation_context</span><span class="p">,</span>
<span class="n">input_bundle</span><span class="p">,</span> <span class="n">fired_timers</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">,</span>
<span class="n">completion_callback</span><span class="p">,</span> <span class="n">transform_evaluation_state</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_evaluator_registry</span> <span class="o">=</span> <span class="n">transform_evaluator_registry</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_evaluation_context</span> <span class="o">=</span> <span class="n">evaluation_context</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_input_bundle</span> <span class="o">=</span> <span class="n">input_bundle</span>
<span class="c1"># For non-empty bundles, store the window of the max EOW.</span>
<span class="c1"># TODO(mariagh): Move to class _Bundle&#39;s inner _StackedWindowedValues</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_latest_main_input_window</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">input_bundle</span><span class="o">.</span><span class="n">has_elements</span><span class="p">():</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_latest_main_input_window</span> <span class="o">=</span> <span class="n">input_bundle</span><span class="o">.</span><span class="n">_elements</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">windows</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">for</span> <span class="n">elem</span> <span class="ow">in</span> <span class="n">input_bundle</span><span class="o">.</span><span class="n">get_elements_iterable</span><span class="p">():</span>
<span class="k">if</span> <span class="n">elem</span><span class="o">.</span><span class="n">windows</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">end</span> <span class="o">&gt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_latest_main_input_window</span><span class="o">.</span><span class="n">end</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_latest_main_input_window</span> <span class="o">=</span> <span class="n">elem</span><span class="o">.</span><span class="n">windows</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_fired_timers</span> <span class="o">=</span> <span class="n">fired_timers</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_applied_ptransform</span> <span class="o">=</span> <span class="n">applied_ptransform</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_completion_callback</span> <span class="o">=</span> <span class="n">completion_callback</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_evaluation_state</span> <span class="o">=</span> <span class="n">transform_evaluation_state</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_side_input_values</span> <span class="o">=</span> <span class="p">{}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">blocked</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_call_count</span> <span class="o">=</span> <span class="mi">0</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_retry_count</span> <span class="o">=</span> <span class="mi">0</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_max_retries_per_bundle</span> <span class="o">=</span> <span class="n">TransformExecutor</span><span class="o">.</span><span class="n">_MAX_RETRY_PER_BUNDLE</span>
<div class="viewcode-block" id="TransformExecutor.call"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.executor.html#apache_beam.runners.direct.executor.TransformExecutor.call">[docs]</a> <span class="k">def</span> <span class="nf">call</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">state_sampler</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_call_count</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="k">assert</span> <span class="bp">self</span><span class="o">.</span><span class="n">_call_count</span> <span class="o">&lt;=</span> <span class="p">(</span><span class="mi">1</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">_applied_ptransform</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">))</span>
<span class="n">metrics_container</span> <span class="o">=</span> <span class="n">MetricsContainer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_applied_ptransform</span><span class="o">.</span><span class="n">full_label</span><span class="p">)</span>
<span class="n">start_state</span> <span class="o">=</span> <span class="n">state_sampler</span><span class="o">.</span><span class="n">scoped_state</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_applied_ptransform</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span>
<span class="s1">&#39;start&#39;</span><span class="p">,</span>
<span class="n">metrics_container</span><span class="o">=</span><span class="n">metrics_container</span><span class="p">)</span>
<span class="n">process_state</span> <span class="o">=</span> <span class="n">state_sampler</span><span class="o">.</span><span class="n">scoped_state</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_applied_ptransform</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span>
<span class="s1">&#39;process&#39;</span><span class="p">,</span>
<span class="n">metrics_container</span><span class="o">=</span><span class="n">metrics_container</span><span class="p">)</span>
<span class="n">finish_state</span> <span class="o">=</span> <span class="n">state_sampler</span><span class="o">.</span><span class="n">scoped_state</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_applied_ptransform</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span>
<span class="s1">&#39;finish&#39;</span><span class="p">,</span>
<span class="n">metrics_container</span><span class="o">=</span><span class="n">metrics_container</span><span class="p">)</span>
<span class="k">with</span> <span class="n">start_state</span><span class="p">:</span>
<span class="c1"># Side input initialization should be accounted for in start_state.</span>
<span class="k">for</span> <span class="n">side_input</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_applied_ptransform</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">:</span>
<span class="c1"># Find the projection of main&#39;s window onto the side input&#39;s window.</span>
<span class="n">window_mapping_fn</span> <span class="o">=</span> <span class="n">side_input</span><span class="o">.</span><span class="n">_view_options</span><span class="p">()</span><span class="o">.</span><span class="n">get</span><span class="p">(</span>
<span class="s1">&#39;window_mapping_fn&#39;</span><span class="p">,</span> <span class="n">sideinputs</span><span class="o">.</span><span class="n">_global_window_mapping_fn</span><span class="p">)</span>
<span class="n">main_onto_side_window</span> <span class="o">=</span> <span class="n">window_mapping_fn</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_latest_main_input_window</span><span class="p">)</span>
<span class="n">block_until</span> <span class="o">=</span> <span class="n">main_onto_side_window</span><span class="o">.</span><span class="n">end</span>
<span class="k">if</span> <span class="n">side_input</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_side_input_values</span><span class="p">:</span>
<span class="n">value</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_evaluation_context</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="bp">self</span><span class="p">,</span> <span class="n">block_until</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">value</span><span class="p">:</span>
<span class="c1"># Monitor task will reschedule this executor once the side input is</span>
<span class="c1"># available.</span>
<span class="k">return</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_side_input_values</span><span class="p">[</span><span class="n">side_input</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span>
<span class="n">side_input_values</span> <span class="o">=</span> <span class="p">[</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_side_input_values</span><span class="p">[</span><span class="n">side_input</span><span class="p">]</span>
<span class="k">for</span> <span class="n">side_input</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_applied_ptransform</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">]</span>
<span class="k">while</span> <span class="bp">self</span><span class="o">.</span><span class="n">_retry_count</span> <span class="o">&lt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_retries_per_bundle</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">attempt_call</span><span class="p">(</span><span class="n">metrics_container</span><span class="p">,</span>
<span class="n">side_input_values</span><span class="p">,</span>
<span class="n">start_state</span><span class="p">,</span>
<span class="n">process_state</span><span class="p">,</span>
<span class="n">finish_state</span><span class="p">)</span>
<span class="k">break</span>
<span class="k">except</span> <span class="ne">Exception</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_retry_count</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="n">logging</span><span class="o">.</span><span class="n">error</span><span class="p">(</span>
<span class="s1">&#39;Exception at bundle </span><span class="si">%r</span><span class="s1">, due to an exception.</span><span class="se">\n</span><span class="s1"> </span><span class="si">%s</span><span class="s1">&#39;</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_input_bundle</span><span class="p">,</span> <span class="n">traceback</span><span class="o">.</span><span class="n">format_exc</span><span class="p">())</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_retry_count</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_retries_per_bundle</span><span class="p">:</span>
<span class="n">logging</span><span class="o">.</span><span class="n">error</span><span class="p">(</span><span class="s1">&#39;Giving up after </span><span class="si">%s</span><span class="s1"> attempts.&#39;</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_max_retries_per_bundle</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_completion_callback</span><span class="o">.</span><span class="n">handle_exception</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">e</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_evaluation_context</span><span class="o">.</span><span class="n">metrics</span><span class="p">()</span><span class="o">.</span><span class="n">commit_physical</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_input_bundle</span><span class="p">,</span>
<span class="n">metrics_container</span><span class="o">.</span><span class="n">get_cumulative</span><span class="p">())</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_evaluation_state</span><span class="o">.</span><span class="n">complete</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span></div>
<div class="viewcode-block" id="TransformExecutor.attempt_call"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.executor.html#apache_beam.runners.direct.executor.TransformExecutor.attempt_call">[docs]</a> <span class="k">def</span> <span class="nf">attempt_call</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">metrics_container</span><span class="p">,</span>
<span class="n">side_input_values</span><span class="p">,</span>
<span class="n">start_state</span><span class="p">,</span>
<span class="n">process_state</span><span class="p">,</span>
<span class="n">finish_state</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Attempts to run a bundle.&quot;&quot;&quot;</span>
<span class="n">evaluator</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_transform_evaluator_registry</span><span class="o">.</span><span class="n">get_evaluator</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_applied_ptransform</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_input_bundle</span><span class="p">,</span>
<span class="n">side_input_values</span><span class="p">)</span>
<span class="k">with</span> <span class="n">start_state</span><span class="p">:</span>
<span class="n">evaluator</span><span class="o">.</span><span class="n">start_bundle</span><span class="p">()</span>
<span class="k">with</span> <span class="n">process_state</span><span class="p">:</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fired_timers</span><span class="p">:</span>
<span class="k">for</span> <span class="n">timer_firing</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_fired_timers</span><span class="p">:</span>
<span class="n">evaluator</span><span class="o">.</span><span class="n">process_timer_wrapper</span><span class="p">(</span><span class="n">timer_firing</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_input_bundle</span><span class="p">:</span>
<span class="k">for</span> <span class="n">value</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_input_bundle</span><span class="o">.</span><span class="n">get_elements_iterable</span><span class="p">():</span>
<span class="n">evaluator</span><span class="o">.</span><span class="n">process_element</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">with</span> <span class="n">finish_state</span><span class="p">:</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">evaluator</span><span class="o">.</span><span class="n">finish_bundle</span><span class="p">()</span>
<span class="n">result</span><span class="o">.</span><span class="n">logical_metric_updates</span> <span class="o">=</span> <span class="n">metrics_container</span><span class="o">.</span><span class="n">get_cumulative</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_completion_callback</span><span class="o">.</span><span class="n">handle_result</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">_input_bundle</span><span class="p">,</span> <span class="n">result</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span></div></div>
<div class="viewcode-block" id="Executor"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.executor.html#apache_beam.runners.direct.executor.Executor">[docs]</a><span class="k">class</span> <span class="nc">Executor</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.&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="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span> <span class="o">=</span> <span class="n">_ExecutorServiceParallelExecutor</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<div class="viewcode-block" id="Executor.start"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.executor.html#apache_beam.runners.direct.executor.Executor.start">[docs]</a> <span class="k">def</span> <span class="nf">start</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">roots</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">start</span><span class="p">(</span><span class="n">roots</span><span class="p">)</span></div>
<div class="viewcode-block" id="Executor.await_completion"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.executor.html#apache_beam.runners.direct.executor.Executor.await_completion">[docs]</a> <span class="k">def</span> <span class="nf">await_completion</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">_executor</span><span class="o">.</span><span class="n">await_completion</span><span class="p">()</span></div>
<div class="viewcode-block" id="Executor.shutdown"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.executor.html#apache_beam.runners.direct.executor.Executor.shutdown">[docs]</a> <span class="k">def</span> <span class="nf">shutdown</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">_executor</span><span class="o">.</span><span class="n">request_shutdown</span><span class="p">()</span></div></div>
<span class="k">class</span> <span class="nc">_ExecutorServiceParallelExecutor</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;An internal implementation for Executor.&quot;&quot;&quot;</span>
<span class="n">NUM_WORKERS</span> <span class="o">=</span> <span class="mi">1</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">value_to_consumers</span><span class="p">,</span> <span class="n">transform_evaluator_registry</span><span class="p">,</span>
<span class="n">evaluation_context</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">executor_service</span> <span class="o">=</span> <span class="n">_ExecutorService</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">NUM_WORKERS</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">transform_executor_services</span> <span class="o">=</span> <span class="n">_TransformExecutorServices</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">executor_service</span><span class="p">)</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">transform_evaluator_registry</span> <span class="o">=</span> <span class="n">transform_evaluator_registry</span>
<span class="bp">self</span><span class="o">.</span><span class="n">evaluation_context</span> <span class="o">=</span> <span class="n">evaluation_context</span>
<span class="bp">self</span><span class="o">.</span><span class="n">all_updates</span> <span class="o">=</span> <span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_TypedUpdateQueue</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_ExecutorUpdate</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">visible_updates</span> <span class="o">=</span> <span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_TypedUpdateQueue</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_VisibleExecutorUpdate</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">default_completion_callback</span> <span class="o">=</span> <span class="n">_CompletionCallback</span><span class="p">(</span>
<span class="n">evaluation_context</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">all_updates</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">start</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">roots</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">root_nodes</span> <span class="o">=</span> <span class="nb">frozenset</span><span class="p">(</span><span class="n">roots</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">all_nodes</span> <span class="o">=</span> <span class="nb">frozenset</span><span class="p">(</span>
<span class="n">itertools</span><span class="o">.</span><span class="n">chain</span><span class="p">(</span>
<span class="n">roots</span><span class="p">,</span>
<span class="o">*</span><span class="n">itertools</span><span class="o">.</span><span class="n">chain</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">value_to_consumers</span><span class="o">.</span><span class="n">values</span><span class="p">())))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">node_to_pending_bundles</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">root_node</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">root_nodes</span><span class="p">:</span>
<span class="n">provider</span> <span class="o">=</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">transform_evaluator_registry</span>
<span class="o">.</span><span class="n">get_root_bundle_provider</span><span class="p">(</span><span class="n">root_node</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">node_to_pending_bundles</span><span class="p">[</span><span class="n">root_node</span><span class="p">]</span> <span class="o">=</span> <span class="n">provider</span><span class="o">.</span><span class="n">get_root_bundles</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">executor_service</span><span class="o">.</span><span class="n">submit</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_MonitorTask</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">await_completion</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">update</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">visible_updates</span><span class="o">.</span><span class="n">take</span><span class="p">()</span>
<span class="k">try</span><span class="p">:</span>
<span class="k">if</span> <span class="n">update</span><span class="o">.</span><span class="n">exception</span><span class="p">:</span>
<span class="n">t</span><span class="p">,</span> <span class="n">v</span><span class="p">,</span> <span class="n">tb</span> <span class="o">=</span> <span class="n">update</span><span class="o">.</span><span class="n">exc_info</span>
<span class="n">raise_</span><span class="p">(</span><span class="n">t</span><span class="p">,</span> <span class="n">v</span><span class="p">,</span> <span class="n">tb</span><span class="p">)</span>
<span class="k">finally</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">executor_service</span><span class="o">.</span><span class="n">shutdown</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">executor_service</span><span class="o">.</span><span class="n">await_completion</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">request_shutdown</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">executor_service</span><span class="o">.</span><span class="n">shutdown</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">schedule_consumers</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">committed_bundle</span><span class="p">):</span>
<span class="k">if</span> <span class="n">committed_bundle</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">value_to_consumers</span><span class="p">:</span>
<span class="n">consumers</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">value_to_consumers</span><span class="p">[</span><span class="n">committed_bundle</span><span class="o">.</span><span class="n">pcollection</span><span class="p">]</span>
<span class="k">for</span> <span class="n">applied_ptransform</span> <span class="ow">in</span> <span class="n">consumers</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">schedule_consumption</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="p">,</span> <span class="n">committed_bundle</span><span class="p">,</span> <span class="p">[],</span>
<span class="bp">self</span><span class="o">.</span><span class="n">default_completion_callback</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">schedule_unprocessed_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">,</span>
<span class="n">unprocessed_bundle</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">node_to_pending_bundles</span><span class="p">[</span><span class="n">applied_ptransform</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">unprocessed_bundle</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">schedule_consumption</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">consumer_applied_ptransform</span><span class="p">,</span> <span class="n">committed_bundle</span><span class="p">,</span>
<span class="n">fired_timers</span><span class="p">,</span> <span class="n">on_complete</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Schedules evaluation of the given bundle with the transform.&quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="n">consumer_applied_ptransform</span>
<span class="k">assert</span> <span class="n">committed_bundle</span>
<span class="k">assert</span> <span class="n">on_complete</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">transform_evaluator_registry</span><span class="o">.</span><span class="n">should_execute_serially</span><span class="p">(</span>
<span class="n">consumer_applied_ptransform</span><span class="p">):</span>
<span class="n">transform_executor_service</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">transform_executor_services</span><span class="o">.</span><span class="n">serial</span><span class="p">(</span>
<span class="n">consumer_applied_ptransform</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">transform_executor_service</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">transform_executor_services</span><span class="o">.</span><span class="n">parallel</span><span class="p">()</span>
<span class="n">transform_executor</span> <span class="o">=</span> <span class="n">TransformExecutor</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">transform_evaluator_registry</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">evaluation_context</span><span class="p">,</span>
<span class="n">committed_bundle</span><span class="p">,</span> <span class="n">fired_timers</span><span class="p">,</span> <span class="n">consumer_applied_ptransform</span><span class="p">,</span>
<span class="n">on_complete</span><span class="p">,</span> <span class="n">transform_executor_service</span><span class="p">)</span>
<span class="n">transform_executor_service</span><span class="o">.</span><span class="n">schedule</span><span class="p">(</span><span class="n">transform_executor</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_TypedUpdateQueue</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Type checking update queue with blocking and non-blocking operations.&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">item_type</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_item_type</span> <span class="o">=</span> <span class="n">item_type</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_queue</span> <span class="o">=</span> <span class="n">queue</span><span class="o">.</span><span class="n">Queue</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">poll</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">item</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_queue</span><span class="o">.</span><span class="n">get_nowait</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_queue</span><span class="o">.</span><span class="n">task_done</span><span class="p">()</span>
<span class="k">return</span> <span class="n">item</span>
<span class="k">except</span> <span class="n">queue</span><span class="o">.</span><span class="n">Empty</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">take</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># The implementation of Queue.Queue.get() does not propagate</span>
<span class="c1"># KeyboardInterrupts when a timeout is not used. We therefore use a</span>
<span class="c1"># one-second timeout in the following loop to allow KeyboardInterrupts</span>
<span class="c1"># to be correctly propagated.</span>
<span class="k">while</span> <span class="kc">True</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">item</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_queue</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">timeout</span><span class="o">=</span><span class="mi">1</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_queue</span><span class="o">.</span><span class="n">task_done</span><span class="p">()</span>
<span class="k">return</span> <span class="n">item</span>
<span class="k">except</span> <span class="n">queue</span><span class="o">.</span><span class="n">Empty</span><span class="p">:</span>
<span class="k">pass</span>
<span class="k">def</span> <span class="nf">offer</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">item</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">item</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_item_type</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_queue</span><span class="o">.</span><span class="n">put_nowait</span><span class="p">(</span><span class="n">item</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_ExecutorUpdate</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;An internal status update on the state of the executor.&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">transform_executor</span><span class="p">,</span> <span class="n">committed_bundle</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">unprocessed_bundle</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">exception</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">transform_executor</span> <span class="o">=</span> <span class="n">transform_executor</span>
<span class="c1"># Exactly one of them should be not-None</span>
<span class="k">assert</span> <span class="nb">sum</span><span class="p">([</span>
<span class="nb">bool</span><span class="p">(</span><span class="n">committed_bundle</span><span class="p">),</span>
<span class="nb">bool</span><span class="p">(</span><span class="n">unprocessed_bundle</span><span class="p">),</span>
<span class="nb">bool</span><span class="p">(</span><span class="n">exception</span><span class="p">)])</span> <span class="o">==</span> <span class="mi">1</span>
<span class="bp">self</span><span class="o">.</span><span class="n">committed_bundle</span> <span class="o">=</span> <span class="n">committed_bundle</span>
<span class="bp">self</span><span class="o">.</span><span class="n">unprocessed_bundle</span> <span class="o">=</span> <span class="n">unprocessed_bundle</span>
<span class="bp">self</span><span class="o">.</span><span class="n">exception</span> <span class="o">=</span> <span class="n">exception</span>
<span class="bp">self</span><span class="o">.</span><span class="n">exc_info</span> <span class="o">=</span> <span class="n">sys</span><span class="o">.</span><span class="n">exc_info</span><span class="p">()</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">exc_info</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">exception</span><span class="p">:</span>
<span class="c1"># Not the right exception.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">exc_info</span> <span class="o">=</span> <span class="p">(</span><span class="n">exception</span><span class="p">,</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_VisibleExecutorUpdate</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;An update of interest to the user.</span>
<span class="sd"> Used for awaiting the completion to decide whether to return normally or</span>
<span class="sd"> raise an exception.</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">exc_info</span><span class="o">=</span><span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">)):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">finished</span> <span class="o">=</span> <span class="n">exc_info</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">exception</span> <span class="o">=</span> <span class="n">exc_info</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span> <span class="ow">or</span> <span class="n">exc_info</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">exc_info</span> <span class="o">=</span> <span class="n">exc_info</span>
<span class="k">class</span> <span class="nc">_MonitorTask</span><span class="p">(</span><span class="n">_ExecutorService</span><span class="o">.</span><span class="n">CallableTask</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;MonitorTask continuously runs to ensure that pipeline makes progress.&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">executor</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span> <span class="o">=</span> <span class="n">executor</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">name</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;monitor&#39;</span>
<span class="k">def</span> <span class="nf">call</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">state_sampler</span><span class="p">):</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">update</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">all_updates</span><span class="o">.</span><span class="n">poll</span><span class="p">()</span>
<span class="k">while</span> <span class="n">update</span><span class="p">:</span>
<span class="k">if</span> <span class="n">update</span><span class="o">.</span><span class="n">committed_bundle</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">schedule_consumers</span><span class="p">(</span><span class="n">update</span><span class="o">.</span><span class="n">committed_bundle</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">update</span><span class="o">.</span><span class="n">unprocessed_bundle</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">schedule_unprocessed_bundle</span><span class="p">(</span>
<span class="n">update</span><span class="o">.</span><span class="n">transform_executor</span><span class="o">.</span><span class="n">_applied_ptransform</span><span class="p">,</span>
<span class="n">update</span><span class="o">.</span><span class="n">unprocessed_bundle</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">assert</span> <span class="n">update</span><span class="o">.</span><span class="n">exception</span>
<span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span><span class="s1">&#39;A task failed with exception: </span><span class="si">%s</span><span class="s1">&#39;</span><span class="p">,</span>
<span class="n">update</span><span class="o">.</span><span class="n">exception</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">visible_updates</span><span class="o">.</span><span class="n">offer</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_VisibleExecutorUpdate</span><span class="p">(</span>
<span class="n">update</span><span class="o">.</span><span class="n">exc_info</span><span class="p">))</span>
<span class="n">update</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">all_updates</span><span class="o">.</span><span class="n">poll</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">evaluation_context</span><span class="o">.</span><span class="n">schedule_pending_unblocked_tasks</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">executor_service</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_add_work_if_necessary</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_fire_timers</span><span class="p">())</span>
<span class="k">except</span> <span class="ne">Exception</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span> <span class="c1"># pylint: disable=broad-except</span>
<span class="n">logging</span><span class="o">.</span><span class="n">error</span><span class="p">(</span><span class="s1">&#39;Monitor task died due to exception.</span><span class="se">\n</span><span class="s1"> </span><span class="si">%s</span><span class="s1">&#39;</span><span class="p">,</span> <span class="n">e</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">visible_updates</span><span class="o">.</span><span class="n">offer</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_VisibleExecutorUpdate</span><span class="p">(</span>
<span class="n">sys</span><span class="o">.</span><span class="n">exc_info</span><span class="p">()))</span>
<span class="k">finally</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">_should_shutdown</span><span class="p">():</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">executor_service</span><span class="o">.</span><span class="n">submit</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_should_shutdown</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Checks whether the pipeline is completed and should be shut down.</span>
<span class="sd"> If there is anything in the queue of tasks to do or</span>
<span class="sd"> if there are any realtime timers set, do not shut down.</span>
<span class="sd"> Otherwise, check if all the transforms&#39; watermarks are complete.</span>
<span class="sd"> If they are not, the pipeline is not progressing (stall detected).</span>
<span class="sd"> Whether the pipeline has stalled or not, the executor should shut</span>
<span class="sd"> down the pipeline.</span>
<span class="sd"> Returns:</span>
<span class="sd"> True only if the pipeline has reached a terminal state and should</span>
<span class="sd"> be shut down.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_executing</span><span class="p">():</span>
<span class="c1"># There are some bundles still in progress.</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="n">watermark_manager</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">evaluation_context</span><span class="o">.</span><span class="n">_watermark_manager</span>
<span class="n">_</span><span class="p">,</span> <span class="n">any_unfired_realtime_timers</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>
<span class="k">if</span> <span class="n">any_unfired_realtime_timers</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">evaluation_context</span><span class="o">.</span><span class="n">is_done</span><span class="p">():</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">visible_updates</span><span class="o">.</span><span class="n">offer</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_VisibleExecutorUpdate</span><span class="p">())</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Nothing is scheduled for execution, but watermarks incomplete.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">visible_updates</span><span class="o">.</span><span class="n">offer</span><span class="p">(</span>
<span class="n">_ExecutorServiceParallelExecutor</span><span class="o">.</span><span class="n">_VisibleExecutorUpdate</span><span class="p">(</span>
<span class="p">(</span><span class="ne">Exception</span><span class="p">(</span><span class="s1">&#39;Monitor task detected a pipeline stall.&#39;</span><span class="p">),</span>
<span class="kc">None</span><span class="p">,</span>
<span class="kc">None</span><span class="p">)))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">executor_service</span><span class="o">.</span><span class="n">shutdown</span><span class="p">()</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">def</span> <span class="nf">_fire_timers</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Schedules triggered consumers if any timers fired.</span>
<span class="sd"> Returns:</span>
<span class="sd"> True if timers fired.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">transform_fired_timers</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">evaluation_context</span><span class="o">.</span><span class="n">extract_all_timers</span><span class="p">())</span>
<span class="k">for</span> <span class="n">applied_ptransform</span><span class="p">,</span> <span class="n">fired_timers</span> <span class="ow">in</span> <span class="n">transform_fired_timers</span><span class="p">:</span>
<span class="c1"># Use an empty committed bundle. just to trigger.</span>
<span class="n">empty_bundle</span> <span class="o">=</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">evaluation_context</span><span class="o">.</span><span class="n">create_empty_committed_bundle</span><span class="p">(</span>
<span class="n">applied_ptransform</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]))</span>
<span class="n">timer_completion_callback</span> <span class="o">=</span> <span class="n">_CompletionCallback</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">evaluation_context</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">all_updates</span><span class="p">,</span>
<span class="n">timer_firings</span><span class="o">=</span><span class="n">fired_timers</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">schedule_consumption</span><span class="p">(</span>
<span class="n">applied_ptransform</span><span class="p">,</span> <span class="n">empty_bundle</span><span class="p">,</span> <span class="n">fired_timers</span><span class="p">,</span>
<span class="n">timer_completion_callback</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">bool</span><span class="p">(</span><span class="n">transform_fired_timers</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_is_executing</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Checks whether the job is still executing.</span>
<span class="sd"> Returns:</span>
<span class="sd"> True if there is at least one non-blocked TransformExecutor active.&quot;&quot;&quot;</span>
<span class="n">executors</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">transform_executor_services</span><span class="o">.</span><span class="n">executors</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">executors</span><span class="p">:</span>
<span class="c1"># Nothing is executing.</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="c1"># Ensure that at least one of those executors is not blocked.</span>
<span class="k">for</span> <span class="n">transform_executor</span> <span class="ow">in</span> <span class="n">executors</span><span class="p">:</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">transform_executor</span><span class="o">.</span><span class="n">blocked</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">def</span> <span class="nf">_add_work_if_necessary</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">timers_fired</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Adds more work from the roots if pipeline requires more input.</span>
<span class="sd"> If all active TransformExecutors are in a blocked state, add more work</span>
<span class="sd"> from root nodes that may have additional work. This ensures that if a</span>
<span class="sd"> pipeline has elements available from the root nodes it will add those</span>
<span class="sd"> elements when necessary.</span>
<span class="sd"> Args:</span>
<span class="sd"> timers_fired: True if any timers fired prior to this call.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># If any timers have fired, they will add more work; No need to add more.</span>
<span class="k">if</span> <span class="n">timers_fired</span><span class="p">:</span>
<span class="k">return</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_executing</span><span class="p">():</span>
<span class="c1"># We have at least one executor that can proceed without adding</span>
<span class="c1"># additional work.</span>
<span class="k">return</span>
<span class="c1"># All current TransformExecutors are blocked; add more work from any</span>
<span class="c1"># pending bundles.</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">_executor</span><span class="o">.</span><span class="n">all_nodes</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">_executor</span><span class="o">.</span><span class="n">evaluation_context</span><span class="o">.</span><span class="n">is_done</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="p">):</span>
<span class="n">pending_bundles</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">node_to_pending_bundles</span><span class="o">.</span><span class="n">get</span><span class="p">(</span>
<span class="n">applied_ptransform</span><span class="p">,</span> <span class="p">[])</span>
<span class="k">for</span> <span class="n">bundle</span> <span class="ow">in</span> <span class="n">pending_bundles</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">schedule_consumption</span><span class="p">(</span>
<span class="n">applied_ptransform</span><span class="p">,</span> <span class="n">bundle</span><span class="p">,</span> <span class="p">[],</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">default_completion_callback</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</span><span class="o">.</span><span class="n">node_to_pending_bundles</span><span class="p">[</span><span class="n">applied_ptransform</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
</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>