blob: 9929896a7a4b86ed86b7a113e3a4c85a4732d39b [file] [log] [blame]
<!DOCTYPE html>
<html class="writer-html5" lang="en" data-content_root="../../../">
<head>
<meta charset="utf-8" />
<meta name="viewport" content="width=device-width, initial-scale=1.0" />
<title>apache_beam.runners.runner &mdash; Apache Beam 2.67.0 documentation</title>
<link rel="stylesheet" type="text/css" href="../../../_static/pygments.css?v=b86133f3" />
<link rel="stylesheet" type="text/css" href="../../../_static/css/theme.css?v=e59714d7" />
<script src="../../../_static/jquery.js?v=5d32c60e"></script>
<script src="../../../_static/_sphinx_javascript_frameworks_compat.js?v=2cd50e6c"></script>
<script src="../../../_static/documentation_options.js?v=959b4fbe"></script>
<script src="../../../_static/doctools.js?v=9a2dae69"></script>
<script src="../../../_static/sphinx_highlight.js?v=dc90522c"></script>
<script src="../../../_static/js/theme.js"></script>
<link rel="index" title="Index" href="../../../genindex.html" />
<link rel="search" title="Search" href="../../../search.html" />
</head>
<body class="wy-body-for-nav">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search" >
<a href="../../../index.html" class="icon icon-home">
Apache Beam
</a>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" aria-label="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="Navigation menu">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.dataframe.html">apache_beam.dataframe package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.ml.html">apache_beam.ml package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.utils.html">apache_beam.utils package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.yaml.html">apache_beam.yaml package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap"><nav class="wy-nav-top" aria-label="Mobile navigation menu" >
<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="Page navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../index.html" class="icon icon-home" aria-label="Home"></a></li>
<li class="breadcrumb-item"><a href="../../index.html">Module code</a></li>
<li class="breadcrumb-item active">apache_beam.runners.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.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;PipelineRunner, an abstract base runner object.&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span><span class="w"> </span><span class="nn">importlib</span>
<span class="kn">import</span><span class="w"> </span><span class="nn">logging</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">typing</span><span class="w"> </span><span class="kn">import</span> <span class="n">TYPE_CHECKING</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">typing</span><span class="w"> </span><span class="kn">import</span> <span class="n">Iterable</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">typing</span><span class="w"> </span><span class="kn">import</span> <span class="n">Optional</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.options.pipeline_options</span><span class="w"> </span><span class="kn">import</span> <span class="n">PipelineOptions</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.options.pipeline_options</span><span class="w"> </span><span class="kn">import</span> <span class="n">PortableOptions</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.options.pipeline_options</span><span class="w"> </span><span class="kn">import</span> <span class="n">SetupOptions</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.options.pipeline_options</span><span class="w"> </span><span class="kn">import</span> <span class="n">StandardOptions</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.options.pipeline_options</span><span class="w"> </span><span class="kn">import</span> <span class="n">TypeOptions</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.portability</span><span class="w"> </span><span class="kn">import</span> <span class="n">common_urns</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.portability.api</span><span class="w"> </span><span class="kn">import</span> <span class="n">beam_runner_api_pb2</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.runners.pipeline_utils</span><span class="w"> </span><span class="kn">import</span> <span class="n">group_by_key_input_visitor</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.transforms</span><span class="w"> </span><span class="kn">import</span> <span class="n">environments</span>
<span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam</span><span class="w"> </span><span class="kn">import</span> <span class="n">pvalue</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam</span><span class="w"> </span><span class="kn">import</span> <span class="n">PTransform</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.pipeline</span><span class="w"> </span><span class="kn">import</span> <span class="n">Pipeline</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s1">&#39;PipelineRunner&#39;</span><span class="p">,</span> <span class="s1">&#39;PipelineState&#39;</span><span class="p">,</span> <span class="s1">&#39;PipelineResult&#39;</span><span class="p">]</span>
<span class="n">_RUNNER_MAP</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">path</span><span class="o">.</span><span class="n">rsplit</span><span class="p">(</span><span class="s1">&#39;.&#39;</span><span class="p">,</span> <span class="n">maxsplit</span><span class="o">=</span><span class="mi">1</span><span class="p">)[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span><span class="o">.</span><span class="n">lower</span><span class="p">():</span> <span class="n">path</span>
<span class="k">for</span> <span class="n">path</span> <span class="ow">in</span> <span class="n">StandardOptions</span><span class="o">.</span><span class="n">ALL_KNOWN_RUNNERS</span>
<span class="p">}</span>
<span class="c1"># Allow this alias, but don&#39;t make public.</span>
<span class="n">_RUNNER_MAP</span><span class="p">[</span><span class="s1">&#39;pythonrpcdirectrunner&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span>
<span class="s1">&#39;apache_beam.runners.experimental&#39;</span>
<span class="s1">&#39;.python_rpc_direct.python_rpc_direct_runner.PythonRPCDirectRunner&#39;</span><span class="p">)</span>
<span class="n">_LOGGER</span> <span class="o">=</span> <span class="n">logging</span><span class="o">.</span><span class="n">getLogger</span><span class="p">(</span><span class="vm">__name__</span><span class="p">)</span>
<span class="k">def</span><span class="w"> </span><span class="nf">create_runner</span><span class="p">(</span><span class="n">runner_name</span><span class="p">:</span> <span class="nb">str</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s1">&#39;PipelineRunner&#39;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> Creates a runner instance from a runner class name.</span>
<span class="sd"> Args:</span>
<span class="sd"> runner_name: Name of the pipeline runner. Possible values are listed in</span>
<span class="sd"> _RUNNER_MAP above.</span>
<span class="sd"> Returns:</span>
<span class="sd"> A runner object.</span>
<span class="sd"> Raises:</span>
<span class="sd"> RuntimeError: if an invalid runner name is used.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Get the qualified runner name by using the lower case runner name. If that</span>
<span class="c1"># fails try appending the name with &#39;runner&#39; and check if it matches.</span>
<span class="c1"># If that also fails, use the given runner name as is.</span>
<span class="n">runner_name</span> <span class="o">=</span> <span class="n">_RUNNER_MAP</span><span class="o">.</span><span class="n">get</span><span class="p">(</span>
<span class="n">runner_name</span><span class="o">.</span><span class="n">lower</span><span class="p">(),</span>
<span class="n">_RUNNER_MAP</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">runner_name</span><span class="o">.</span><span class="n">lower</span><span class="p">()</span> <span class="o">+</span> <span class="s1">&#39;runner&#39;</span><span class="p">,</span> <span class="n">runner_name</span><span class="p">))</span>
<span class="k">if</span> <span class="s1">&#39;.&#39;</span> <span class="ow">in</span> <span class="n">runner_name</span><span class="p">:</span>
<span class="n">module</span><span class="p">,</span> <span class="n">runner</span> <span class="o">=</span> <span class="n">runner_name</span><span class="o">.</span><span class="n">rsplit</span><span class="p">(</span><span class="s1">&#39;.&#39;</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">importlib</span><span class="o">.</span><span class="n">import_module</span><span class="p">(</span><span class="n">module</span><span class="p">),</span> <span class="n">runner</span><span class="p">)()</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="k">if</span> <span class="s1">&#39;dataflow&#39;</span> <span class="ow">in</span> <span class="n">runner_name</span><span class="o">.</span><span class="n">lower</span><span class="p">():</span>
<span class="k">raise</span> <span class="ne">ImportError</span><span class="p">(</span>
<span class="s1">&#39;Google Cloud Dataflow runner not available, &#39;</span>
<span class="s1">&#39;please install apache_beam[gcp]&#39;</span><span class="p">)</span>
<span class="k">elif</span> <span class="s1">&#39;interactive&#39;</span> <span class="ow">in</span> <span class="n">runner_name</span><span class="o">.</span><span class="n">lower</span><span class="p">():</span>
<span class="k">raise</span> <span class="ne">ImportError</span><span class="p">(</span>
<span class="s1">&#39;Interactive runner not available, &#39;</span>
<span class="s1">&#39;please install apache_beam[interactive]&#39;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Unexpected pipeline runner: </span><span class="si">%s</span><span class="s1">. Valid values are </span><span class="si">%s</span><span class="s1"> &#39;</span>
<span class="s1">&#39;or the fully qualified name of a PipelineRunner subclass.&#39;</span> <span class="o">%</span>
<span class="p">(</span><span class="n">runner_name</span><span class="p">,</span> <span class="s1">&#39;, &#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">StandardOptions</span><span class="o">.</span><span class="n">KNOWN_RUNNER_NAMES</span><span class="p">)))</span>
<div class="viewcode-block" id="PipelineRunner">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner">[docs]</a>
<span class="k">class</span><span class="w"> </span><span class="nc">PipelineRunner</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A runner of a pipeline object.</span>
<span class="sd"> The base runner provides a run() method for visiting every node in the</span>
<span class="sd"> pipeline&#39;s DAG and executing the transforms computing the PValue in the node.</span>
<span class="sd"> A custom runner will typically provide implementations for some of the</span>
<span class="sd"> transform methods (ParDo, GroupByKey, Create, etc.). It may also</span>
<span class="sd"> provide a new implementation for clear_pvalue(), which is used to wipe out</span>
<span class="sd"> materialized values in order to reduce footprint.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="PipelineRunner.run">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.run">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">run</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="s1">&#39;PTransform&#39;</span><span class="p">,</span>
<span class="n">options</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">PipelineOptions</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s1">&#39;PipelineResult&#39;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Run the given transform or callable with this runner.</span>
<span class="sd"> Blocks until the pipeline is complete. See also `PipelineRunner.run_async`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">run_async</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span>
<span class="n">result</span><span class="o">.</span><span class="n">wait_until_finish</span><span class="p">()</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="PipelineRunner.run_async">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.run_async">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">run_async</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="s1">&#39;PTransform&#39;</span><span class="p">,</span>
<span class="n">options</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">PipelineOptions</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s1">&#39;PipelineResult&#39;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Run the given transform or callable with this runner.</span>
<span class="sd"> May return immediately, executing the pipeline in the background.</span>
<span class="sd"> The returned result object can be queried for progress, and</span>
<span class="sd"> `wait_until_finish` may be called to block until completion.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Imported here to avoid circular dependencies.</span>
<span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam</span><span class="w"> </span><span class="kn">import</span> <span class="n">PTransform</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.pvalue</span><span class="w"> </span><span class="kn">import</span> <span class="n">PBegin</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.pipeline</span><span class="w"> </span><span class="kn">import</span> <span class="n">Pipeline</span>
<span class="n">p</span> <span class="o">=</span> <span class="n">Pipeline</span><span class="p">(</span><span class="n">runner</span><span class="o">=</span><span class="bp">self</span><span class="p">,</span> <span class="n">options</span><span class="o">=</span><span class="n">options</span><span class="p">)</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">PTransform</span><span class="p">):</span>
<span class="n">p</span> <span class="o">|</span> <span class="n">transform</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">transform</span><span class="p">(</span><span class="n">PBegin</span><span class="p">(</span><span class="n">p</span><span class="p">))</span>
<span class="k">return</span> <span class="n">p</span><span class="o">.</span><span class="n">run</span><span class="p">()</span></div>
<div class="viewcode-block" id="PipelineRunner.run_portable_pipeline">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.run_portable_pipeline">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">run_portable_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">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">,</span>
<span class="n">options</span><span class="p">:</span> <span class="n">PipelineOptions</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s1">&#39;PipelineResult&#39;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Execute the entire pipeline.</span>
<span class="sd"> Runners should override this method.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="PipelineRunner.default_environment">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.default_environment">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">default_environment</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">options</span><span class="p">:</span> <span class="n">PipelineOptions</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">environments</span><span class="o">.</span><span class="n">Environment</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns the default environment that should be used for this runner.</span>
<span class="sd"> Runners may override this method to provide alternative environments.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">environments</span><span class="o">.</span><span class="n">Environment</span><span class="o">.</span><span class="n">from_options</span><span class="p">(</span>
<span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">PortableOptions</span><span class="p">))</span></div>
<div class="viewcode-block" id="PipelineRunner.run_pipeline">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.run_pipeline">[docs]</a>
<span class="k">def</span><span class="w"> </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="s1">&#39;Pipeline&#39;</span><span class="p">,</span> <span class="n">options</span><span class="p">:</span> <span class="n">PipelineOptions</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s1">&#39;PipelineResult&#39;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Execute the entire pipeline or the sub-DAG reachable from a node.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span>
<span class="n">group_by_key_input_visitor</span><span class="p">(</span>
<span class="ow">not</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">TypeOptions</span><span class="p">)</span><span class="o">.</span><span class="n">allow_non_deterministic_key_coders</span><span class="p">)</span>
<span class="p">)</span>
<span class="c1"># TODO: https://github.com/apache/beam/issues/19168</span>
<span class="c1"># portable runner specific default</span>
<span class="k">if</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">SetupOptions</span><span class="p">)</span><span class="o">.</span><span class="n">sdk_location</span> <span class="o">==</span> <span class="s1">&#39;default&#39;</span><span class="p">:</span>
<span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">SetupOptions</span><span class="p">)</span><span class="o">.</span><span class="n">sdk_location</span> <span class="o">=</span> <span class="s1">&#39;container&#39;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">run_portable_pipeline</span><span class="p">(</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">(</span>
<span class="n">default_environment</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">default_environment</span><span class="p">(</span><span class="n">options</span><span class="p">)),</span>
<span class="n">options</span><span class="p">)</span></div>
<div class="viewcode-block" id="PipelineRunner.apply">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.apply">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">apply</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="s1">&#39;PTransform&#39;</span><span class="p">,</span>
<span class="nb">input</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="s1">&#39;pvalue.PValue&#39;</span><span class="p">],</span>
<span class="n">options</span><span class="p">:</span> <span class="n">PipelineOptions</span><span class="p">):</span>
<span class="c1"># TODO(robertwb): Remove indirection once internal references are fixed.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">apply_PTransform</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="nb">input</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span></div>
<div class="viewcode-block" id="PipelineRunner.apply_PTransform">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.apply_PTransform">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">apply_PTransform</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="nb">input</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># TODO(robertwb): Remove indirection once internal references are fixed.</span>
<span class="k">return</span> <span class="n">transform</span><span class="o">.</span><span class="n">expand</span><span class="p">(</span><span class="nb">input</span><span class="p">)</span></div>
<div class="viewcode-block" id="PipelineRunner.is_fnapi_compatible">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.is_fnapi_compatible">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">is_fnapi_compatible</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Whether to enable the beam_fn_api experiment by default.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="kc">True</span></div>
<div class="viewcode-block" id="PipelineRunner.check_requirements">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.check_requirements">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">check_requirements</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">pipeline_proto</span><span class="p">:</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">,</span>
<span class="n">supported_requirements</span><span class="p">:</span> <span class="n">Iterable</span><span class="p">[</span><span class="nb">str</span><span class="p">]):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Check that this runner can satisfy all pipeline requirements.&quot;&quot;&quot;</span>
<span class="c1"># Imported here to avoid circular dependencies.</span>
<span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span>
<span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.runners.portability.fn_api_runner</span><span class="w"> </span><span class="kn">import</span> <span class="n">translations</span>
<span class="n">supported_requirements</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">supported_requirements</span><span class="p">)</span>
<span class="k">for</span> <span class="n">requirement</span> <span class="ow">in</span> <span class="n">pipeline_proto</span><span class="o">.</span><span class="n">requirements</span><span class="p">:</span>
<span class="k">if</span> <span class="n">requirement</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">supported_requirements</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Unable to run pipeline with requirement: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">requirement</span><span class="p">)</span>
<span class="k">for</span> <span class="n">transform</span> <span class="ow">in</span> <span class="n">pipeline_proto</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">TEST_STREAM</span><span class="o">.</span><span class="n">urn</span><span class="p">:</span>
<span class="k">if</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">TEST_STREAM</span><span class="o">.</span><span class="n">urn</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">supported_requirements</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">transform</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span> <span class="ow">in</span> <span class="n">translations</span><span class="o">.</span><span class="n">PAR_DO_URNS</span><span class="p">:</span>
<span class="n">payload</span> <span class="o">=</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ParDoPayload</span><span class="o">.</span><span class="n">FromString</span><span class="p">(</span>
<span class="n">transform</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">payload</span><span class="p">)</span>
<span class="k">for</span> <span class="n">timer</span> <span class="ow">in</span> <span class="n">payload</span><span class="o">.</span><span class="n">timer_family_specs</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="k">if</span> <span class="n">timer</span><span class="o">.</span><span class="n">time_domain</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">(</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">TimeDomain</span><span class="o">.</span><span class="n">EVENT_TIME</span><span class="p">,</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">TimeDomain</span><span class="o">.</span><span class="n">PROCESSING_TIME</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="n">timer</span><span class="o">.</span><span class="n">time_domain</span><span class="p">)</span></div>
<div class="viewcode-block" id="PipelineRunner.default_pickle_library_override">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.default_pickle_library_override">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">default_pickle_library_override</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Default pickle library, can be overridden by runner implementation.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="kc">None</span></div>
</div>
<span class="c1"># FIXME: replace with PipelineState(str, enum.Enum)</span>
<div class="viewcode-block" id="PipelineState">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineState">[docs]</a>
<span class="k">class</span><span class="w"> </span><span class="nc">PipelineState</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;State of the Pipeline, as returned by :attr:`PipelineResult.state`.</span>
<span class="sd"> This is meant to be the union of all the states any runner can put a</span>
<span class="sd"> pipeline in. Currently, it represents the values of the dataflow</span>
<span class="sd"> API JobState enum.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">UNKNOWN</span> <span class="o">=</span> <span class="s1">&#39;UNKNOWN&#39;</span> <span class="c1"># not specified by a runner, or unknown to a runner.</span>
<span class="n">STARTING</span> <span class="o">=</span> <span class="s1">&#39;STARTING&#39;</span> <span class="c1"># not yet started</span>
<span class="n">STOPPED</span> <span class="o">=</span> <span class="s1">&#39;STOPPED&#39;</span> <span class="c1"># paused or not yet started</span>
<span class="n">RUNNING</span> <span class="o">=</span> <span class="s1">&#39;RUNNING&#39;</span> <span class="c1"># currently running</span>
<span class="n">DONE</span> <span class="o">=</span> <span class="s1">&#39;DONE&#39;</span> <span class="c1"># successfully completed (terminal state)</span>
<span class="n">FAILED</span> <span class="o">=</span> <span class="s1">&#39;FAILED&#39;</span> <span class="c1"># failed (terminal state)</span>
<span class="n">CANCELLED</span> <span class="o">=</span> <span class="s1">&#39;CANCELLED&#39;</span> <span class="c1"># explicitly cancelled (terminal state)</span>
<span class="n">UPDATED</span> <span class="o">=</span> <span class="s1">&#39;UPDATED&#39;</span> <span class="c1"># replaced by another job (terminal state)</span>
<span class="n">DRAINING</span> <span class="o">=</span> <span class="s1">&#39;DRAINING&#39;</span> <span class="c1"># still processing, no longer reading data</span>
<span class="n">DRAINED</span> <span class="o">=</span> <span class="s1">&#39;DRAINED&#39;</span> <span class="c1"># draining completed (terminal state)</span>
<span class="n">PENDING</span> <span class="o">=</span> <span class="s1">&#39;PENDING&#39;</span> <span class="c1"># the job has been created but is not yet running.</span>
<span class="n">CANCELLING</span> <span class="o">=</span> <span class="s1">&#39;CANCELLING&#39;</span> <span class="c1"># job has been explicitly cancelled and is</span>
<span class="c1"># in the process of stopping</span>
<span class="n">RESOURCE_CLEANING_UP</span> <span class="o">=</span> <span class="s1">&#39;RESOURCE_CLEANING_UP&#39;</span> <span class="c1"># job&#39;s resources are being</span>
<span class="c1"># cleaned up</span>
<span class="n">UNRECOGNIZED</span> <span class="o">=</span> <span class="s1">&#39;UNRECOGNIZED&#39;</span> <span class="c1"># the job state reported by a runner cannot be</span>
<span class="c1"># interpreted by the SDK.</span>
<div class="viewcode-block" id="PipelineState.is_terminal">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineState.is_terminal">[docs]</a>
<span class="nd">@classmethod</span>
<span class="k">def</span><span class="w"> </span><span class="nf">is_terminal</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">state</span><span class="p">):</span>
<span class="k">return</span> <span class="n">state</span> <span class="ow">in</span> <span class="p">[</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">DONE</span><span class="p">,</span> <span class="bp">cls</span><span class="o">.</span><span class="n">FAILED</span><span class="p">,</span> <span class="bp">cls</span><span class="o">.</span><span class="n">CANCELLED</span><span class="p">,</span> <span class="bp">cls</span><span class="o">.</span><span class="n">UPDATED</span><span class="p">,</span> <span class="bp">cls</span><span class="o">.</span><span class="n">DRAINED</span>
<span class="p">]</span></div>
</div>
<div class="viewcode-block" id="PipelineResult">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineResult">[docs]</a>
<span class="k">class</span><span class="w"> </span><span class="nc">PipelineResult</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A :class:`PipelineResult` provides access to info about a pipeline.&quot;&quot;&quot;</span>
<span class="k">def</span><span class="w"> </span><span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">state</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_state</span> <span class="o">=</span> <span class="n">state</span>
<span class="nd">@property</span>
<span class="k">def</span><span class="w"> </span><span class="nf">state</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Return the current state of the pipeline execution.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_state</span>
<div class="viewcode-block" id="PipelineResult.wait_until_finish">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineResult.wait_until_finish">[docs]</a>
<span class="k">def</span><span class="w"> </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="c1"># pylint: disable=unused-argument</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Waits until the pipeline finishes and returns the final status.</span>
<span class="sd"> Args:</span>
<span class="sd"> duration (int): The time to wait (in milliseconds) for job to finish.</span>
<span class="sd"> If it is set to :data:`None`, it will wait indefinitely until the job</span>
<span class="sd"> is finished.</span>
<span class="sd"> Raises:</span>
<span class="sd"> IOError: If there is a persistent problem getting job</span>
<span class="sd"> information.</span>
<span class="sd"> NotImplementedError: If the runner does not support this</span>
<span class="sd"> operation.</span>
<span class="sd"> Returns:</span>
<span class="sd"> The final state of the pipeline, or :data:`None` on timeout.</span>
<span class="sd"> &quot;&quot;&quot;</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">raise</span> <span class="ne">NotImplementedError</span><span class="p">()</span></div>
<div class="viewcode-block" id="PipelineResult.cancel">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineResult.cancel">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">cancel</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Cancels the pipeline execution.</span>
<span class="sd"> Raises:</span>
<span class="sd"> IOError: If there is a persistent problem getting job</span>
<span class="sd"> information.</span>
<span class="sd"> NotImplementedError: If the runner does not support this</span>
<span class="sd"> operation.</span>
<span class="sd"> Returns:</span>
<span class="sd"> The final state of the pipeline.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">()</span></div>
<div class="viewcode-block" id="PipelineResult.metrics">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineResult.metrics">[docs]</a>
<span class="k">def</span><span class="w"> </span><span class="nf">metrics</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns :class:`~apache_beam.metrics.metric.MetricResults` object to</span>
<span class="sd"> query metrics from the runner.</span>
<span class="sd"> Raises:</span>
<span class="sd"> NotImplementedError: If the runner does not support this</span>
<span class="sd"> operation.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">()</span></div>
<span class="c1"># pylint: disable=unused-argument</span>
<div class="viewcode-block" id="PipelineResult.aggregated_values">
<a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineResult.aggregated_values">[docs]</a>
<span class="k">def</span><span class="w"> </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="w"> </span><span class="sd">&quot;&quot;&quot;Return a dict of step names to values of the Aggregator.&quot;&quot;&quot;</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;</span><span class="si">%s</span><span class="s1"> does not implement aggregated_values&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span><span class="p">)</span>
<span class="k">return</span> <span class="p">{}</span></div>
</div>
</pre></div>
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>&#169; Copyright %Y, Apache Beam.</p>
</div>
Built with <a href="https://www.sphinx-doc.org/">Sphinx</a> using a
<a href="https://github.com/readthedocs/sphinx_rtd_theme">theme</a>
provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script>
jQuery(function () {
SphinxRtdTheme.Navigation.enable(true);
});
</script>
</body>
</html>