blob: 920c4b6b8d6c1ccbe7cebe9244ad656731f94058 [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.direct_runner &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.direct_runner</li>
<li class="wy-breadcrumbs-aside">
</li>
</ul>
<hr/>
</div>
<div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article">
<div itemprop="articleBody">
<h1>Source code for apache_beam.runners.direct.direct_runner</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="sd">&quot;&quot;&quot;DirectRunner, executing on the local machine.</span>
<span class="sd">The DirectRunner is a runner implementation that executes the entire</span>
<span class="sd">graph of transformations belonging to a pipeline on the local machine.</span>
<span class="sd">&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">itertools</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">import</span> <span class="nn">time</span>
<span class="kn">import</span> <span class="nn">typing</span>
<span class="kn">from</span> <span class="nn">google.protobuf</span> <span class="k">import</span> <span class="n">wrappers_pb2</span>
<span class="kn">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">coders</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">typehints</span>
<span class="kn">from</span> <span class="nn">apache_beam.internal.util</span> <span class="k">import</span> <span class="n">ArgumentPlaceholder</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">DirectOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">StandardOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.value_provider</span> <span class="k">import</span> <span class="n">RuntimeValueProvider</span>
<span class="kn">from</span> <span class="nn">apache_beam.pvalue</span> <span class="k">import</span> <span class="n">PCollection</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.bundle_factory</span> <span class="k">import</span> <span class="n">BundleFactory</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.clock</span> <span class="k">import</span> <span class="n">RealClock</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.clock</span> <span class="k">import</span> <span class="n">TestClock</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.runner</span> <span class="k">import</span> <span class="n">PipelineResult</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.runner</span> <span class="k">import</span> <span class="n">PipelineRunner</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.runner</span> <span class="k">import</span> <span class="n">PipelineState</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">CombinePerKey</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">CombineValuesDoFn</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">DoFn</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">ParDo</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">_GroupAlsoByWindow</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">_GroupAlsoByWindowDoFn</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">_GroupByKeyOnly</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="k">import</span> <span class="n">PTransform</span>
<span class="c1"># Note that the BundleBasedDirectRunner and SwitchingDirectRunner names are</span>
<span class="c1"># experimental and have no backwards compatibility guarantees.</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s1">&#39;BundleBasedDirectRunner&#39;</span><span class="p">,</span>
<span class="s1">&#39;DirectRunner&#39;</span><span class="p">,</span>
<span class="s1">&#39;SwitchingDirectRunner&#39;</span><span class="p">]</span>
<div class="viewcode-block" id="SwitchingDirectRunner"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.direct_runner.html#apache_beam.runners.direct.direct_runner.SwitchingDirectRunner">[docs]</a><span class="k">class</span> <span class="nc">SwitchingDirectRunner</span><span class="p">(</span><span class="n">PipelineRunner</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Executes a single pipeline on the local machine.</span>
<span class="sd"> This implementation switches between using the FnApiRunner (which has</span>
<span class="sd"> high throughput for batch jobs) and using the BundleBasedDirectRunner,</span>
<span class="sd"> which supports streaming execution and certain primitives not yet</span>
<span class="sd"> implemented in the FnApiRunner.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="SwitchingDirectRunner.run_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.direct_runner.html#apache_beam.runners.direct.direct_runner.SwitchingDirectRunner.run_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">run_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PipelineVisitor</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.native_io.iobase</span> <span class="k">import</span> <span class="n">NativeSource</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.native_io.iobase</span> <span class="k">import</span> <span class="n">_NativeWrite</span>
<span class="kn">from</span> <span class="nn">apache_beam.testing.test_stream</span> <span class="k">import</span> <span class="n">TestStream</span>
<span class="k">class</span> <span class="nc">_FnApiRunnerSupportVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Visitor determining if a Pipeline can be run on the FnApiRunner.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">accept</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">supported_by_fnapi_runner</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">visit</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">supported_by_fnapi_runner</span>
<span class="k">def</span> <span class="nf">visit_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="n">transform</span> <span class="o">=</span> <span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span>
<span class="c1"># The FnApiRunner does not support streaming execution.</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">TestStream</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">supported_by_fnapi_runner</span> <span class="o">=</span> <span class="kc">False</span>
<span class="c1"># The FnApiRunner does not support reads from NativeSources.</span>
<span class="k">if</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">Read</span><span class="p">)</span> <span class="ow">and</span>
<span class="nb">isinstance</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">,</span> <span class="n">NativeSource</span><span class="p">)):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">supported_by_fnapi_runner</span> <span class="o">=</span> <span class="kc">False</span>
<span class="c1"># The FnApiRunner does not support the use of _NativeWrites.</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">_NativeWrite</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">supported_by_fnapi_runner</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">):</span>
<span class="n">dofn</span> <span class="o">=</span> <span class="n">transform</span><span class="o">.</span><span class="n">dofn</span>
<span class="c1"># The FnApiRunner does not support execution of CombineFns with</span>
<span class="c1"># deferred side inputs.</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">dofn</span><span class="p">,</span> <span class="n">CombineValuesDoFn</span><span class="p">):</span>
<span class="n">args</span><span class="p">,</span> <span class="n">kwargs</span> <span class="o">=</span> <span class="n">transform</span><span class="o">.</span><span class="n">raw_side_inputs</span>
<span class="n">args_to_check</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="n">args</span><span class="p">,</span>
<span class="n">kwargs</span><span class="o">.</span><span class="n">values</span><span class="p">())</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">arg</span><span class="p">,</span> <span class="n">ArgumentPlaceholder</span><span class="p">)</span>
<span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">args_to_check</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">supported_by_fnapi_runner</span> <span class="o">=</span> <span class="kc">False</span>
<span class="c1"># Check whether all transforms used in the pipeline are supported by the</span>
<span class="c1"># FnApiRunner, and the pipeline was not meant to be run as streaming.</span>
<span class="n">use_fnapi_runner</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">_FnApiRunnerSupportVisitor</span><span class="p">()</span><span class="o">.</span><span class="n">accept</span><span class="p">(</span><span class="n">pipeline</span><span class="p">))</span>
<span class="c1"># Also ensure grpc is available.</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># pylint: disable=unused-variable</span>
<span class="kn">import</span> <span class="nn">grpc</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="n">use_fnapi_runner</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">if</span> <span class="n">use_fnapi_runner</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.portability.fn_api_runner</span> <span class="k">import</span> <span class="n">FnApiRunner</span>
<span class="n">runner</span> <span class="o">=</span> <span class="n">FnApiRunner</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">runner</span> <span class="o">=</span> <span class="n">BundleBasedDirectRunner</span><span class="p">()</span>
<span class="k">return</span> <span class="n">runner</span><span class="o">.</span><span class="n">run_pipeline</span><span class="p">(</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span></div></div>
<span class="c1"># Type variables.</span>
<span class="n">K</span> <span class="o">=</span> <span class="n">typing</span><span class="o">.</span><span class="n">TypeVar</span><span class="p">(</span><span class="s1">&#39;K&#39;</span><span class="p">)</span>
<span class="n">V</span> <span class="o">=</span> <span class="n">typing</span><span class="o">.</span><span class="n">TypeVar</span><span class="p">(</span><span class="s1">&#39;V&#39;</span><span class="p">)</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">typing</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">typing</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">typing</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">V</span><span class="p">]])</span>
<span class="k">class</span> <span class="nc">_StreamingGroupByKeyOnly</span><span class="p">(</span><span class="n">_GroupByKeyOnly</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Streaming GroupByKeyOnly placeholder for overriding in DirectRunner.&quot;&quot;&quot;</span>
<span class="n">urn</span> <span class="o">=</span> <span class="s2">&quot;direct_runner:streaming_gbko:v0.1&quot;</span>
<span class="c1"># These are needed due to apply overloads.</span>
<span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_StreamingGroupByKeyOnly</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span>
<span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_payload</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_StreamingGroupByKeyOnly</span><span class="p">()</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">typing</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">typing</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">V</span><span class="p">]])</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">typing</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">typing</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">V</span><span class="p">]])</span>
<span class="k">class</span> <span class="nc">_StreamingGroupAlsoByWindow</span><span class="p">(</span><span class="n">_GroupAlsoByWindow</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Streaming GroupAlsoByWindow placeholder for overriding in DirectRunner.&quot;&quot;&quot;</span>
<span class="n">urn</span> <span class="o">=</span> <span class="s2">&quot;direct_runner:streaming_gabw:v0.1&quot;</span>
<span class="c1"># These are needed due to apply overloads.</span>
<span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">_StreamingGroupAlsoByWindow</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">wrappers_pb2</span><span class="o">.</span><span class="n">BytesValue</span><span class="p">(</span><span class="n">value</span><span class="o">=</span><span class="n">context</span><span class="o">.</span><span class="n">windowing_strategies</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">windowing</span><span class="p">)))</span>
<span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">urn</span><span class="p">,</span> <span class="n">wrappers_pb2</span><span class="o">.</span><span class="n">BytesValue</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">payload</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_StreamingGroupAlsoByWindow</span><span class="p">(</span>
<span class="n">context</span><span class="o">.</span><span class="n">windowing_strategies</span><span class="o">.</span><span class="n">get_by_id</span><span class="p">(</span><span class="n">payload</span><span class="o">.</span><span class="n">value</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">_get_transform_overrides</span><span class="p">(</span><span class="n">pipeline_options</span><span class="p">):</span>
<span class="c1"># A list of PTransformOverride objects to be applied before running a pipeline</span>
<span class="c1"># using DirectRunner.</span>
<span class="c1"># Currently this only works for overrides where the input and output types do</span>
<span class="c1"># not change.</span>
<span class="c1"># For internal use only; no backwards-compatibility guarantees.</span>
<span class="c1"># Importing following locally to avoid a circular dependency.</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PTransformOverride</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.sdf_common</span> <span class="k">import</span> <span class="n">SplittableParDoOverride</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.helper_transforms</span> <span class="k">import</span> <span class="n">LiftedCombinePerKey</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.sdf_direct_runner</span> <span class="k">import</span> <span class="n">ProcessKeyedElementsViaKeyedWorkItemsOverride</span>
<span class="k">class</span> <span class="nc">CombinePerKeyOverride</span><span class="p">(</span><span class="n">PTransformOverride</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">matches</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="n">CombinePerKey</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">def</span> <span class="nf">get_replacement_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">):</span>
<span class="c1"># TODO: Move imports to top. Pipeline &lt;-&gt; Runner dependency cause problems</span>
<span class="c1"># with resolving imports when they are at top.</span>
<span class="c1"># pylint: disable=wrong-import-position</span>
<span class="k">try</span><span class="p">:</span>
<span class="k">return</span> <span class="n">LiftedCombinePerKey</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">args</span><span class="p">,</span>
<span class="n">transform</span><span class="o">.</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">NotImplementedError</span><span class="p">:</span>
<span class="k">return</span> <span class="n">transform</span>
<span class="k">class</span> <span class="nc">StreamingGroupByKeyOverride</span><span class="p">(</span><span class="n">PTransformOverride</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">matches</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="c1"># Note: we match the exact class, since we replace it with a subclass.</span>
<span class="k">return</span> <span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="vm">__class__</span> <span class="o">==</span> <span class="n">_GroupByKeyOnly</span>
<span class="k">def</span> <span class="nf">get_replacement_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">):</span>
<span class="c1"># Use specialized streaming implementation.</span>
<span class="n">transform</span> <span class="o">=</span> <span class="n">_StreamingGroupByKeyOnly</span><span class="p">()</span>
<span class="k">return</span> <span class="n">transform</span>
<span class="k">class</span> <span class="nc">StreamingGroupAlsoByWindowOverride</span><span class="p">(</span><span class="n">PTransformOverride</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">matches</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="c1"># Note: we match the exact class, since we replace it with a subclass.</span>
<span class="n">transform</span> <span class="o">=</span> <span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span>
<span class="k">return</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="n">ParDo</span><span class="p">)</span> <span class="ow">and</span>
<span class="nb">isinstance</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">dofn</span><span class="p">,</span> <span class="n">_GroupAlsoByWindowDoFn</span><span class="p">)</span> <span class="ow">and</span>
<span class="n">transform</span><span class="o">.</span><span class="vm">__class__</span> <span class="o">!=</span> <span class="n">_StreamingGroupAlsoByWindow</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_replacement_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">):</span>
<span class="c1"># Use specialized streaming implementation.</span>
<span class="n">transform</span> <span class="o">=</span> <span class="n">_StreamingGroupAlsoByWindow</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">dofn</span><span class="o">.</span><span class="n">windowing</span><span class="p">)</span>
<span class="k">return</span> <span class="n">transform</span>
<span class="n">overrides</span> <span class="o">=</span> <span class="p">[</span><span class="n">SplittableParDoOverride</span><span class="p">(),</span>
<span class="n">ProcessKeyedElementsViaKeyedWorkItemsOverride</span><span class="p">(),</span>
<span class="n">CombinePerKeyOverride</span><span class="p">()]</span>
<span class="c1"># Add streaming overrides, if necessary.</span>
<span class="k">if</span> <span class="n">pipeline_options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span><span class="o">.</span><span class="n">streaming</span><span class="p">:</span>
<span class="n">overrides</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">StreamingGroupByKeyOverride</span><span class="p">())</span>
<span class="n">overrides</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">StreamingGroupAlsoByWindowOverride</span><span class="p">())</span>
<span class="c1"># Add PubSub overrides, if PubSub is available.</span>
<span class="k">try</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">apache_beam.io.gcp</span> <span class="k">import</span> <span class="n">pubsub</span> <span class="k">as</span> <span class="n">unused_pubsub</span>
<span class="n">overrides</span> <span class="o">+=</span> <span class="n">_get_pubsub_transform_overrides</span><span class="p">(</span><span class="n">pipeline_options</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="k">pass</span>
<span class="k">return</span> <span class="n">overrides</span>
<span class="k">class</span> <span class="nc">_DirectReadFromPubSub</span><span class="p">(</span><span class="n">PTransform</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">source</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_source</span> <span class="o">=</span> <span class="n">source</span>
<span class="k">def</span> <span class="nf">_infer_output_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_input_type</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">unused_input_coder</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">return</span> <span class="n">coders</span><span class="o">.</span><span class="n">BytesCoder</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">get_windowing</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">inputs</span><span class="p">):</span>
<span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">Windowing</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalue</span><span class="p">):</span>
<span class="c1"># This is handled as a native transform.</span>
<span class="k">return</span> <span class="n">PCollection</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">_DirectWriteToPubSubFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span>
<span class="n">BUFFER_SIZE_ELEMENTS</span> <span class="o">=</span> <span class="mi">100</span>
<span class="n">FLUSH_TIMEOUT_SECS</span> <span class="o">=</span> <span class="n">BUFFER_SIZE_ELEMENTS</span> <span class="o">*</span> <span class="mf">0.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">sink</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">project</span> <span class="o">=</span> <span class="n">sink</span><span class="o">.</span><span class="n">project</span>
<span class="bp">self</span><span class="o">.</span><span class="n">short_topic_name</span> <span class="o">=</span> <span class="n">sink</span><span class="o">.</span><span class="n">topic_name</span>
<span class="bp">self</span><span class="o">.</span><span class="n">id_label</span> <span class="o">=</span> <span class="n">sink</span><span class="o">.</span><span class="n">id_label</span>
<span class="bp">self</span><span class="o">.</span><span class="n">timestamp_attribute</span> <span class="o">=</span> <span class="n">sink</span><span class="o">.</span><span class="n">timestamp_attribute</span>
<span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span> <span class="o">=</span> <span class="n">sink</span><span class="o">.</span><span class="n">with_attributes</span>
<span class="c1"># TODO(BEAM-4275): Add support for id_label and timestamp_attribute.</span>
<span class="k">if</span> <span class="n">sink</span><span class="o">.</span><span class="n">id_label</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;DirectRunner: id_label is not supported for &#39;</span>
<span class="s1">&#39;PubSub writes&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">sink</span><span class="o">.</span><span class="n">timestamp_attribute</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;DirectRunner: timestamp_attribute is not &#39;</span>
<span class="s1">&#39;supported for PubSub writes&#39;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">start_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">elem</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">elem</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="p">)</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">BUFFER_SIZE_ELEMENTS</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_flush</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_flush</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_flush</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="kn">from</span> <span class="nn">google.cloud</span> <span class="k">import</span> <span class="n">pubsub</span>
<span class="n">pub_client</span> <span class="o">=</span> <span class="n">pubsub</span><span class="o">.</span><span class="n">PublisherClient</span><span class="p">()</span>
<span class="n">topic</span> <span class="o">=</span> <span class="n">pub_client</span><span class="o">.</span><span class="n">topic_path</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">project</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">short_topic_name</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">with_attributes</span><span class="p">:</span>
<span class="n">futures</span> <span class="o">=</span> <span class="p">[</span><span class="n">pub_client</span><span class="o">.</span><span class="n">publish</span><span class="p">(</span><span class="n">topic</span><span class="p">,</span> <span class="n">elem</span><span class="o">.</span><span class="n">data</span><span class="p">,</span> <span class="o">**</span><span class="n">elem</span><span class="o">.</span><span class="n">attributes</span><span class="p">)</span>
<span class="k">for</span> <span class="n">elem</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">futures</span> <span class="o">=</span> <span class="p">[</span><span class="n">pub_client</span><span class="o">.</span><span class="n">publish</span><span class="p">(</span><span class="n">topic</span><span class="p">,</span> <span class="n">elem</span><span class="p">)</span>
<span class="k">for</span> <span class="n">elem</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="p">]</span>
<span class="n">timer_start</span> <span class="o">=</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span>
<span class="k">for</span> <span class="n">future</span> <span class="ow">in</span> <span class="n">futures</span><span class="p">:</span>
<span class="n">remaining</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">FLUSH_TIMEOUT_SECS</span> <span class="o">-</span> <span class="p">(</span><span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> <span class="o">-</span> <span class="n">timer_start</span><span class="p">)</span>
<span class="n">future</span><span class="o">.</span><span class="n">result</span><span class="p">(</span><span class="n">remaining</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">def</span> <span class="nf">_get_pubsub_transform_overrides</span><span class="p">(</span><span class="n">pipeline_options</span><span class="p">):</span>
<span class="kn">from</span> <span class="nn">apache_beam.io.gcp</span> <span class="k">import</span> <span class="n">pubsub</span> <span class="k">as</span> <span class="n">beam_pubsub</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PTransformOverride</span>
<span class="k">class</span> <span class="nc">ReadFromPubSubOverride</span><span class="p">(</span><span class="n">PTransformOverride</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">matches</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span>
<span class="n">beam_pubsub</span><span class="o">.</span><span class="n">ReadFromPubSub</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">get_replacement_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">pipeline_options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span><span class="o">.</span><span class="n">streaming</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">Exception</span><span class="p">(</span><span class="s1">&#39;PubSub I/O is only available in streaming mode &#39;</span>
<span class="s1">&#39;(use the --streaming flag).&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_DirectReadFromPubSub</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">_source</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">WriteToPubSubOverride</span><span class="p">(</span><span class="n">PTransformOverride</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">matches</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span>
<span class="p">(</span><span class="n">beam_pubsub</span><span class="o">.</span><span class="n">WriteToPubSub</span><span class="p">,</span> <span class="n">beam_pubsub</span><span class="o">.</span><span class="n">_WriteStringsToPubSub</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">get_replacement_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">pipeline_options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span><span class="o">.</span><span class="n">streaming</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">Exception</span><span class="p">(</span><span class="s1">&#39;PubSub I/O is only available in streaming mode &#39;</span>
<span class="s1">&#39;(use the --streaming flag).&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="n">_DirectWriteToPubSubFn</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">_sink</span><span class="p">))</span>
<span class="k">return</span> <span class="p">[</span><span class="n">ReadFromPubSubOverride</span><span class="p">(),</span> <span class="n">WriteToPubSubOverride</span><span class="p">()]</span>
<div class="viewcode-block" id="BundleBasedDirectRunner"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.direct_runner.html#apache_beam.runners.direct.direct_runner.BundleBasedDirectRunner">[docs]</a><span class="k">class</span> <span class="nc">BundleBasedDirectRunner</span><span class="p">(</span><span class="n">PipelineRunner</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Executes a single pipeline on the local machine.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="BundleBasedDirectRunner.run_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.direct.direct_runner.html#apache_beam.runners.direct.direct_runner.BundleBasedDirectRunner.run_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">run_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Execute the entire pipeline and returns an DirectPipelineResult.&quot;&quot;&quot;</span>
<span class="c1"># TODO: Move imports to top. Pipeline &lt;-&gt; Runner dependency cause problems</span>
<span class="c1"># with resolving imports when they are at top.</span>
<span class="c1"># pylint: disable=wrong-import-position</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PipelineVisitor</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.consumer_tracking_pipeline_visitor</span> <span class="k">import</span> \
<span class="n">ConsumerTrackingPipelineVisitor</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.evaluation_context</span> <span class="k">import</span> <span class="n">EvaluationContext</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.executor</span> <span class="k">import</span> <span class="n">Executor</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct.transform_evaluator</span> <span class="k">import</span> \
<span class="n">TransformEvaluatorRegistry</span>
<span class="kn">from</span> <span class="nn">apache_beam.testing.test_stream</span> <span class="k">import</span> <span class="n">TestStream</span>
<span class="c1"># Performing configured PTransform overrides.</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">replace_all</span><span class="p">(</span><span class="n">_get_transform_overrides</span><span class="p">(</span><span class="n">options</span><span class="p">))</span>
<span class="c1"># If the TestStream I/O is used, use a mock test clock.</span>
<span class="k">class</span> <span class="nc">_TestStreamUsageVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Visitor determining whether a Pipeline uses a TestStream.&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="bp">self</span><span class="o">.</span><span class="n">uses_test_stream</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">def</span> <span class="nf">visit_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">applied_ptransform</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">applied_ptransform</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="n">TestStream</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">uses_test_stream</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">visitor</span> <span class="o">=</span> <span class="n">_TestStreamUsageVisitor</span><span class="p">()</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="n">visitor</span><span class="p">)</span>
<span class="n">clock</span> <span class="o">=</span> <span class="n">TestClock</span><span class="p">()</span> <span class="k">if</span> <span class="n">visitor</span><span class="o">.</span><span class="n">uses_test_stream</span> <span class="k">else</span> <span class="n">RealClock</span><span class="p">()</span>
<span class="c1"># TODO(BEAM-4274): Circular import runners-metrics. Requires refactoring.</span>
<span class="kn">from</span> <span class="nn">apache_beam.metrics.execution</span> <span class="k">import</span> <span class="n">MetricsEnvironment</span>
<span class="n">MetricsEnvironment</span><span class="o">.</span><span class="n">set_metrics_supported</span><span class="p">(</span><span class="kc">True</span><span class="p">)</span>
<span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">&#39;Running pipeline with DirectRunner.&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">consumer_tracking_visitor</span> <span class="o">=</span> <span class="n">ConsumerTrackingPipelineVisitor</span><span class="p">()</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">consumer_tracking_visitor</span><span class="p">)</span>
<span class="n">evaluation_context</span> <span class="o">=</span> <span class="n">EvaluationContext</span><span class="p">(</span>
<span class="n">options</span><span class="p">,</span>
<span class="n">BundleFactory</span><span class="p">(</span><span class="n">stacked</span><span class="o">=</span><span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">DirectOptions</span><span class="p">)</span>
<span class="o">.</span><span class="n">direct_runner_use_stacked_bundle</span><span class="p">),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">consumer_tracking_visitor</span><span class="o">.</span><span class="n">root_transforms</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">consumer_tracking_visitor</span><span class="o">.</span><span class="n">value_to_consumers</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">consumer_tracking_visitor</span><span class="o">.</span><span class="n">step_names</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">consumer_tracking_visitor</span><span class="o">.</span><span class="n">views</span><span class="p">,</span>
<span class="n">clock</span><span class="p">)</span>
<span class="n">executor</span> <span class="o">=</span> <span class="n">Executor</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">consumer_tracking_visitor</span><span class="o">.</span><span class="n">value_to_consumers</span><span class="p">,</span>
<span class="n">TransformEvaluatorRegistry</span><span class="p">(</span><span class="n">evaluation_context</span><span class="p">),</span>
<span class="n">evaluation_context</span><span class="p">)</span>
<span class="c1"># DirectRunner does not support injecting</span>
<span class="c1"># PipelineOptions values at runtime</span>
<span class="n">RuntimeValueProvider</span><span class="o">.</span><span class="n">set_runtime_options</span><span class="p">({})</span>
<span class="c1"># Start the executor. This is a non-blocking call, it will start the</span>
<span class="c1"># execution in background threads and return.</span>
<span class="n">executor</span><span class="o">.</span><span class="n">start</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">consumer_tracking_visitor</span><span class="o">.</span><span class="n">root_transforms</span><span class="p">)</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">DirectPipelineResult</span><span class="p">(</span><span class="n">executor</span><span class="p">,</span> <span class="n">evaluation_context</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span></div></div>
<span class="c1"># Use the SwitchingDirectRunner as the default.</span>
<span class="n">DirectRunner</span> <span class="o">=</span> <span class="n">SwitchingDirectRunner</span>
<span class="k">class</span> <span class="nc">DirectPipelineResult</span><span class="p">(</span><span class="n">PipelineResult</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A DirectPipelineResult provides access to info about a pipeline.&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="n">evaluation_context</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">DirectPipelineResult</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">PipelineState</span><span class="o">.</span><span class="n">RUNNING</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="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="k">def</span> <span class="nf">__del__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_state</span> <span class="o">==</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">RUNNING</span><span class="p">:</span>
<span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;The DirectPipelineResult is being garbage-collected while the &#39;</span>
<span class="s1">&#39;DirectRunner is still running the corresponding pipeline. This may &#39;</span>
<span class="s1">&#39;lead to incomplete execution of the pipeline if the main thread &#39;</span>
<span class="s1">&#39;exits before pipeline completion. Consider using &#39;</span>
<span class="s1">&#39;result.wait_until_finish() to wait for completion of pipeline &#39;</span>
<span class="s1">&#39;execution.&#39;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">wait_until_finish</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">duration</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">is_terminal</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">):</span>
<span class="k">if</span> <span class="n">duration</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span>
<span class="s1">&#39;DirectRunner does not support duration argument.&#39;</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">_executor</span><span class="o">.</span><span class="n">await_completion</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_state</span> <span class="o">=</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">DONE</span>
<span class="k">except</span><span class="p">:</span> <span class="c1"># pylint: disable=broad-except</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_state</span> <span class="o">=</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">FAILED</span>
<span class="k">raise</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_state</span>
<span class="k">def</span> <span class="nf">aggregated_values</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">aggregator_or_name</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_evaluation_context</span><span class="o">.</span><span class="n">get_aggregator_values</span><span class="p">(</span><span class="n">aggregator_or_name</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">metrics</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</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="k">def</span> <span class="nf">cancel</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Shuts down pipeline workers.</span>
<span class="sd"> For testing use only. Does not properly wait for pipeline workers to shut</span>
<span class="sd"> down.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_state</span> <span class="o">=</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">CANCELLING</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_executor</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">_state</span> <span class="o">=</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">CANCELLED</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>