| |
| |
| <!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.runner — 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> »</li> |
| |
| <li><a href="../../index.html">Module code</a> »</li> |
| |
| <li>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 "License"); 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 "AS IS" 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">"""PipelineRunner, an abstract base runner object."""</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">logging</span> |
| <span class="kn">import</span> <span class="nn">os</span> |
| <span class="kn">import</span> <span class="nn">shelve</span> |
| <span class="kn">import</span> <span class="nn">shutil</span> |
| <span class="kn">import</span> <span class="nn">tempfile</span> |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s1">'PipelineRunner'</span><span class="p">,</span> <span class="s1">'PipelineState'</span><span class="p">,</span> <span class="s1">'PipelineResult'</span><span class="p">]</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_get_runner_map</span><span class="p">(</span><span class="n">runner_names</span><span class="p">,</span> <span class="n">module_path</span><span class="p">):</span> |
| <span class="sd">"""Create a map of runner name in lower case to full import path to the</span> |
| <span class="sd"> runner class.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</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">module_path</span> <span class="o">+</span> <span class="n">runner_name</span> |
| <span class="k">for</span> <span class="n">runner_name</span> <span class="ow">in</span> <span class="n">runner_names</span><span class="p">}</span> |
| |
| |
| <span class="n">_DIRECT_RUNNER_PATH</span> <span class="o">=</span> <span class="s1">'apache_beam.runners.direct.direct_runner.'</span> |
| <span class="n">_DATAFLOW_RUNNER_PATH</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s1">'apache_beam.runners.dataflow.dataflow_runner.'</span><span class="p">)</span> |
| <span class="n">_TEST_RUNNER_PATH</span> <span class="o">=</span> <span class="s1">'apache_beam.runners.test.'</span> |
| <span class="n">_PYTHON_RPC_DIRECT_RUNNER</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s1">'apache_beam.runners.experimental.python_rpc_direct.'</span> |
| <span class="s1">'python_rpc_direct_runner.'</span><span class="p">)</span> |
| |
| <span class="n">_KNOWN_PYTHON_RPC_DIRECT_RUNNER</span> <span class="o">=</span> <span class="p">(</span><span class="s1">'PythonRPCDirectRunner'</span><span class="p">,)</span> |
| <span class="n">_KNOWN_DIRECT_RUNNERS</span> <span class="o">=</span> <span class="p">(</span><span class="s1">'DirectRunner'</span><span class="p">,</span> <span class="s1">'BundleBasedDirectRunner'</span><span class="p">,</span> |
| <span class="s1">'SwitchingDirectRunner'</span><span class="p">)</span> |
| <span class="n">_KNOWN_DATAFLOW_RUNNERS</span> <span class="o">=</span> <span class="p">(</span><span class="s1">'DataflowRunner'</span><span class="p">,)</span> |
| <span class="n">_KNOWN_TEST_RUNNERS</span> <span class="o">=</span> <span class="p">(</span><span class="s1">'TestDataflowRunner'</span><span class="p">,)</span> |
| |
| <span class="n">_RUNNER_MAP</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="n">_RUNNER_MAP</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">_get_runner_map</span><span class="p">(</span><span class="n">_KNOWN_DIRECT_RUNNERS</span><span class="p">,</span> |
| <span class="n">_DIRECT_RUNNER_PATH</span><span class="p">))</span> |
| <span class="n">_RUNNER_MAP</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">_get_runner_map</span><span class="p">(</span><span class="n">_KNOWN_DATAFLOW_RUNNERS</span><span class="p">,</span> |
| <span class="n">_DATAFLOW_RUNNER_PATH</span><span class="p">))</span> |
| <span class="n">_RUNNER_MAP</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">_get_runner_map</span><span class="p">(</span><span class="n">_KNOWN_PYTHON_RPC_DIRECT_RUNNER</span><span class="p">,</span> |
| <span class="n">_PYTHON_RPC_DIRECT_RUNNER</span><span class="p">))</span> |
| <span class="n">_RUNNER_MAP</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">_get_runner_map</span><span class="p">(</span><span class="n">_KNOWN_TEST_RUNNERS</span><span class="p">,</span> |
| <span class="n">_TEST_RUNNER_PATH</span><span class="p">))</span> |
| |
| <span class="n">_ALL_KNOWN_RUNNERS</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">_KNOWN_DIRECT_RUNNERS</span> <span class="o">+</span> <span class="n">_KNOWN_DATAFLOW_RUNNERS</span> <span class="o">+</span> <span class="n">_KNOWN_TEST_RUNNERS</span><span class="p">)</span> |
| |
| |
| <span class="k">def</span> <span class="nf">create_runner</span><span class="p">(</span><span class="n">runner_name</span><span class="p">):</span> |
| <span class="sd">"""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:</span> |
| <span class="sd"> DirectRunner, DataflowRunner and TestDataflowRunner.</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"> """</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 'runner' 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">'runner'</span><span class="p">,</span> <span class="n">runner_name</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="s1">'.'</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">'.'</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="nb">__import__</span><span class="p">(</span><span class="n">module</span><span class="p">,</span> <span class="p">{},</span> <span class="p">{},</span> <span class="p">[</span><span class="n">runner</span><span class="p">],</span> <span class="o">-</span><span class="mi">1</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="n">runner_name</span> <span class="ow">in</span> <span class="n">_KNOWN_DATAFLOW_RUNNERS</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ImportError</span><span class="p">(</span> |
| <span class="s1">'Google Cloud Dataflow runner not available, '</span> |
| <span class="s1">'please install apache_beam[gcp]'</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">'Unexpected pipeline runner: </span><span class="si">%s</span><span class="s1">. Valid values are </span><span class="si">%s</span><span class="s1"> '</span> |
| <span class="s1">'or the fully qualified name of a PipelineRunner subclass.'</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="n">runner_name</span><span class="p">,</span> <span class="s1">', '</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">_ALL_KNOWN_RUNNERS</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="nc">PipelineRunner</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="sd">"""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'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"> """</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="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="n">options</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="sd">"""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"> """</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="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="n">options</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="sd">"""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"> """</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="nn">apache_beam</span> <span class="k">import</span> <span class="n">PTransform</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pvalue</span> <span class="k">import</span> <span class="n">PBegin</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">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_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="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="sd">"""Execute the entire pipeline or the sub-DAG reachable from a node.</span> |
| |
| <span class="sd"> Runners should override this method.</span> |
| <span class="sd"> """</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="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PipelineVisitor</span> |
| |
| <span class="k">class</span> <span class="nc">RunVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</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">runner</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">runner</span> <span class="o">=</span> <span class="n">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">transform_node</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">runner</span><span class="o">.</span><span class="n">run_transform</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| <span class="k">except</span><span class="p">:</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">error</span><span class="p">(</span><span class="s1">'Error while visiting </span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">)</span> |
| <span class="k">raise</span> |
| |
| <span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="n">RunVisitor</span><span class="p">(</span><span class="bp">self</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="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="nb">input</span><span class="p">):</span> |
| <span class="sd">"""Runner callback for a pipeline.apply call.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> transform: the transform to apply.</span> |
| <span class="sd"> input: transform's input (typically a PCollection).</span> |
| |
| <span class="sd"> A concrete implementation of the Runner class may want to do custom</span> |
| <span class="sd"> pipeline construction for a given transform. To override the behavior</span> |
| <span class="sd"> for a transform class Xyz, implement an apply_Xyz method with this same</span> |
| <span class="sd"> signature.</span> |
| <span class="sd"> """</span> |
| <span class="k">for</span> <span class="bp">cls</span> <span class="ow">in</span> <span class="n">transform</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="n">mro</span><span class="p">():</span> |
| <span class="n">m</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s1">'apply_</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="bp">cls</span><span class="o">.</span><span class="vm">__name__</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">m</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">m</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="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="s1">'Execution of [</span><span class="si">%s</span><span class="s1">] not implemented in runner </span><span class="si">%s</span><span class="s1">.'</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="bp">self</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="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="c1"># The base case of apply is to call the transform's expand.</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.run_transform"><a class="viewcode-back" href="../../../apache_beam.runners.runner.html#apache_beam.runners.runner.PipelineRunner.run_transform">[docs]</a> <span class="k">def</span> <span class="nf">run_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">):</span> |
| <span class="sd">"""Runner callback for a pipeline.run call.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> transform_node: transform node for the transform to run.</span> |
| |
| <span class="sd"> A concrete implementation of the Runner class must implement run_Abc for</span> |
| <span class="sd"> some class Abc in the method resolution order for every non-composite</span> |
| <span class="sd"> transform Xyz in the pipeline.</span> |
| <span class="sd"> """</span> |
| <span class="k">for</span> <span class="bp">cls</span> <span class="ow">in</span> <span class="n">transform_node</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">mro</span><span class="p">():</span> |
| <span class="n">m</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s1">'run_</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="bp">cls</span><span class="o">.</span><span class="vm">__name__</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">m</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">m</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="s1">'Execution of [</span><span class="si">%s</span><span class="s1">] not implemented in runner </span><span class="si">%s</span><span class="s1">.'</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="bp">self</span><span class="p">))</span></div></div> |
| |
| |
| <span class="k">class</span> <span class="nc">PValueCache</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="sd">"""For internal use only; no backwards-compatibility guarantees.</span> |
| |
| <span class="sd"> Local cache for arbitrary information computed for PValue objects."""</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">use_disk_backed_cache</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="c1"># Cache of values computed while a runner executes a pipeline. This is a</span> |
| <span class="c1"># dictionary of PValues and their computed values. Note that in principle</span> |
| <span class="c1"># the runner could contain PValues from several pipelines without clashes</span> |
| <span class="c1"># since a PValue is associated with one and only one pipeline. The keys of</span> |
| <span class="c1"># the dictionary are tuple of PValue instance addresses obtained using id()</span> |
| <span class="c1"># and tag names converted to strings.</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_use_disk_backed_cache</span> <span class="o">=</span> <span class="n">use_disk_backed_cache</span> |
| <span class="k">if</span> <span class="n">use_disk_backed_cache</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_tempdir</span> <span class="o">=</span> <span class="n">tempfile</span><span class="o">.</span><span class="n">mkdtemp</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span> <span class="o">=</span> <span class="n">shelve</span><span class="o">.</span><span class="n">open</span><span class="p">(</span><span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_tempdir</span><span class="p">,</span> <span class="s1">'shelve'</span><span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span> <span class="o">=</span> <span class="p">{}</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">_use_disk_backed_cache</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">close</span><span class="p">()</span> |
| <span class="n">shutil</span><span class="o">.</span><span class="n">rmtree</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_tempdir</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">__len__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">to_cache_key</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">,</span> <span class="n">tag</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">transform</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">tag</span> |
| |
| <span class="k">def</span> <span class="nf">_ensure_pvalue_has_real_producer</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="sd">"""Ensure the passed-in PValue has the real_producer attribute.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> pvalue: A PValue instance whose cached value is requested.</span> |
| |
| <span class="sd"> During the runner's execution only the results of the primitive transforms</span> |
| <span class="sd"> are cached. Whenever we are looking for a PValue that is the output of a</span> |
| <span class="sd"> composite transform we need to find the output of its rightmost transform</span> |
| <span class="sd"> part.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">pvalue</span><span class="p">,</span> <span class="s1">'real_producer'</span><span class="p">):</span> |
| <span class="n">real_producer</span> <span class="o">=</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">producer</span> |
| <span class="k">while</span> <span class="n">real_producer</span><span class="o">.</span><span class="n">parts</span><span class="p">:</span> |
| <span class="n">real_producer</span> <span class="o">=</span> <span class="n">real_producer</span><span class="o">.</span><span class="n">parts</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> |
| <span class="n">pvalue</span><span class="o">.</span><span class="n">real_producer</span> <span class="o">=</span> <span class="n">real_producer</span> |
| |
| <span class="k">def</span> <span class="nf">is_cached</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pobj</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">AppliedPTransform</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pobj</span><span class="p">,</span> <span class="n">AppliedPTransform</span><span class="p">):</span> |
| <span class="n">transform</span> <span class="o">=</span> <span class="n">pobj</span> |
| <span class="n">tag</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ensure_pvalue_has_real_producer</span><span class="p">(</span><span class="n">pobj</span><span class="p">)</span> |
| <span class="n">transform</span> <span class="o">=</span> <span class="n">pobj</span><span class="o">.</span><span class="n">real_producer</span> |
| <span class="n">tag</span> <span class="o">=</span> <span class="n">pobj</span><span class="o">.</span><span class="n">tag</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_cache_key</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span> |
| |
| <span class="k">def</span> <span class="nf">cache_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">,</span> <span class="n">tag_or_value</span><span class="p">,</span> <span class="n">value</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">value</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">value</span> <span class="o">=</span> <span class="n">tag_or_value</span> |
| <span class="n">tag</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">tag</span> <span class="o">=</span> <span class="n">tag_or_value</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="p">[</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">to_cache_key</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">tag</span><span class="p">)]</span> <span class="o">=</span> <span class="n">value</span> |
| |
| <span class="k">def</span> <span class="nf">get_pvalue</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="sd">"""Gets the value associated with a PValue from the cache."""</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ensure_pvalue_has_real_producer</span><span class="p">(</span><span class="n">pvalue</span><span class="p">)</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">key</span><span class="p">(</span><span class="n">pvalue</span><span class="p">)]</span> |
| <span class="k">except</span> <span class="ne">KeyError</span><span class="p">:</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">pvalue</span><span class="o">.</span><span class="n">tag</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> |
| <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_cache_key</span><span class="p">(</span><span class="n">pvalue</span><span class="o">.</span><span class="n">real_producer</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="p">):</span> |
| <span class="c1"># This is an undeclared, empty output of a DoFn executed</span> |
| <span class="c1"># in the local runner before this output was referenced.</span> |
| <span class="k">return</span> <span class="p">[]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> |
| |
| <span class="k">def</span> <span class="nf">get_unwindowed_pvalue</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="k">return</span> <span class="p">[</span><span class="n">v</span><span class="o">.</span><span class="n">value</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_pvalue</span><span class="p">(</span><span class="n">pvalue</span><span class="p">)]</span> |
| |
| <span class="k">def</span> <span class="nf">clear_pvalue</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="sd">"""Removes a PValue from the cache."""</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_cached</span><span class="p">(</span><span class="n">pvalue</span><span class="p">):</span> |
| <span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">key</span><span class="p">(</span><span class="n">pvalue</span><span class="p">)]</span> |
| |
| <span class="k">def</span> <span class="nf">key</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pobj</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ensure_pvalue_has_real_producer</span><span class="p">(</span><span class="n">pobj</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_cache_key</span><span class="p">(</span><span class="n">pobj</span><span class="o">.</span><span class="n">real_producer</span><span class="p">,</span> <span class="n">pobj</span><span class="o">.</span><span class="n">tag</span><span class="p">)</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="nc">PipelineState</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="sd">"""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"> """</span> |
| <span class="n">UNKNOWN</span> <span class="o">=</span> <span class="s1">'UNKNOWN'</span> <span class="c1"># not specified</span> |
| <span class="n">STARTING</span> <span class="o">=</span> <span class="s1">'STARTING'</span> <span class="c1"># not yet started</span> |
| <span class="n">STOPPED</span> <span class="o">=</span> <span class="s1">'STOPPED'</span> <span class="c1"># paused or not yet started</span> |
| <span class="n">RUNNING</span> <span class="o">=</span> <span class="s1">'RUNNING'</span> <span class="c1"># currently running</span> |
| <span class="n">DONE</span> <span class="o">=</span> <span class="s1">'DONE'</span> <span class="c1"># successfully completed (terminal state)</span> |
| <span class="n">FAILED</span> <span class="o">=</span> <span class="s1">'FAILED'</span> <span class="c1"># failed (terminal state)</span> |
| <span class="n">CANCELLED</span> <span class="o">=</span> <span class="s1">'CANCELLED'</span> <span class="c1"># explicitly cancelled (terminal state)</span> |
| <span class="n">UPDATED</span> <span class="o">=</span> <span class="s1">'UPDATED'</span> <span class="c1"># replaced by another job (terminal state)</span> |
| <span class="n">DRAINING</span> <span class="o">=</span> <span class="s1">'DRAINING'</span> <span class="c1"># still processing, no longer reading data</span> |
| <span class="n">DRAINED</span> <span class="o">=</span> <span class="s1">'DRAINED'</span> <span class="c1"># draining completed (terminal state)</span> |
| <span class="n">PENDING</span> <span class="o">=</span> <span class="s1">'PENDING'</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">'CANCELLING'</span> <span class="c1"># job has been explicitly cancelled and is</span></div> |
| <span class="c1"># in the process of stopping</span> |
| |
| |
| <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="nc">PipelineResult</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="sd">"""A :class:`PipelineResult` provides access to info about a pipeline."""</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">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="nf">state</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="sd">"""Return the current state of the pipeline execution."""</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="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="sd">"""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"> ~exceptions.IOError: If there is a persistent problem getting job</span> |
| <span class="sd"> information.</span> |
| <span class="sd"> ~exceptions.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"> """</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</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="nf">cancel</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="sd">"""Cancels the pipeline execution.</span> |
| |
| <span class="sd"> Raises:</span> |
| <span class="sd"> ~exceptions.IOError: If there is a persistent problem getting job</span> |
| <span class="sd"> information.</span> |
| <span class="sd"> ~exceptions.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"> """</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</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="nf">metrics</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="sd">"""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"> ~exceptions.NotImplementedError: If the runner does not support this</span> |
| <span class="sd"> operation.</span> |
| <span class="sd"> """</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</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="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="sd">"""Return a dict of step names to values of the Aggregator."""</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span><span class="s1">'</span><span class="si">%s</span><span class="s1"> does not implement aggregated_values'</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 class="articleComments"> |
| |
| </div> |
| </div> |
| <footer> |
| |
| |
| <hr/> |
| |
| <div role="contentinfo"> |
| <p> |
| © 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> |