blob: b3ebf91153323d394b7531fb7bcfa3d4c86e759f [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.runners.interactive.interactive_runner &mdash; Apache Beam documentation</title>
<link rel="stylesheet" href="../../../../_static/css/theme.css" type="text/css" />
<link rel="index" title="Index"
href="../../../../genindex.html"/>
<link rel="search" title="Search" href="../../../../search.html"/>
<link rel="top" title="Apache Beam documentation" href="../../../../index.html"/>
<link rel="up" title="Module code" href="../../../index.html"/>
<script src="../../../../_static/js/modernizr.min.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search">
<a href="../../../../index.html" class="icon icon-home"> Apache Beam
</a>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.internal.html">apache_beam.internal package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.tools.html">apache_beam.tools package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.utils.html">apache_beam.utils package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.version.html">apache_beam.version module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../../index.html">Docs</a> &raquo;</li>
<li><a href="../../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.runners.interactive.interactive_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.interactive.interactive_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;A runner that allows running of Beam pipelines interactively.</span>
<span class="sd">This module is experimental. No backwards-compatibility guarantees.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">division</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">print_function</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">runners</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct</span> <span class="k">import</span> <span class="n">direct_runner</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="k">import</span> <span class="n">cache_manager</span> <span class="k">as</span> <span class="n">cache</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="k">import</span> <span class="n">pipeline_analyzer</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.display</span> <span class="k">import</span> <span class="n">display_manager</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.display</span> <span class="k">import</span> <span class="n">pipeline_graph_renderer</span>
<span class="c1"># size of PCollection samples cached.</span>
<span class="n">SAMPLE_SIZE</span> <span class="o">=</span> <span class="mi">8</span>
<div class="viewcode-block" id="InteractiveRunner"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner">[docs]</a><span class="k">class</span> <span class="nc">InteractiveRunner</span><span class="p">(</span><span class="n">runners</span><span class="o">.</span><span class="n">PipelineRunner</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;An interactive runner for Beam Python pipelines.</span>
<span class="sd"> Allows interactively building and running Beam Python pipelines.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">underlying_runner</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">cache_dir</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">render_option</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Constructor of InteractiveRunner.</span>
<span class="sd"> Args:</span>
<span class="sd"> underlying_runner: (runner.PipelineRunner)</span>
<span class="sd"> cache_dir: (str) the directory where PCollection caches are kept</span>
<span class="sd"> render_option: (str) this parameter decides how the pipeline graph is</span>
<span class="sd"> rendered. See display.pipeline_graph_renderer for available options.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span> <span class="o">=</span> <span class="p">(</span><span class="n">underlying_runner</span>
<span class="ow">or</span> <span class="n">direct_runner</span><span class="o">.</span><span class="n">DirectRunner</span><span class="p">())</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span> <span class="o">=</span> <span class="n">cache</span><span class="o">.</span><span class="n">FileBasedCacheManager</span><span class="p">(</span><span class="n">cache_dir</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_renderer</span> <span class="o">=</span> <span class="n">pipeline_graph_renderer</span><span class="o">.</span><span class="n">get_renderer</span><span class="p">(</span><span class="n">render_option</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span> <span class="o">=</span> <span class="kc">False</span>
<div class="viewcode-block" id="InteractiveRunner.set_render_option"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.set_render_option">[docs]</a> <span class="k">def</span> <span class="nf">set_render_option</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">render_option</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Sets the rendering option.</span>
<span class="sd"> Args:</span>
<span class="sd"> render_option: (str) this parameter decides how the pipeline graph is</span>
<span class="sd"> rendered. See display.pipeline_graph_renderer for available options.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_renderer</span> <span class="o">=</span> <span class="n">pipeline_graph_renderer</span><span class="o">.</span><span class="n">get_renderer</span><span class="p">(</span><span class="n">render_option</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveRunner.start_session"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.start_session">[docs]</a> <span class="k">def</span> <span class="nf">start_session</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Start the session that keeps back-end managers and workers alive.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span><span class="p">:</span>
<span class="k">return</span>
<span class="n">enter</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span> <span class="s1">&#39;__enter__&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">if</span> <span class="n">enter</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">&#39;Starting session.&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">enter</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">logging</span><span class="o">.</span><span class="n">error</span><span class="p">(</span><span class="s1">&#39;Keep alive not supported.&#39;</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveRunner.end_session"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.end_session">[docs]</a> <span class="k">def</span> <span class="nf">end_session</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;End the session that keeps backend managers and workers alive.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span><span class="p">:</span>
<span class="k">return</span>
<span class="n">exit</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span> <span class="s1">&#39;__exit__&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">if</span> <span class="n">exit</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">&#39;Ending session.&#39;</span><span class="p">)</span>
<span class="n">exit</span><span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveRunner.cleanup"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.cleanup">[docs]</a> <span class="k">def</span> <span class="nf">cleanup</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="o">.</span><span class="n">cleanup</span><span class="p">()</span></div>
<div class="viewcode-block" id="InteractiveRunner.apply"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.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="n">pvalueish</span><span class="p">):</span>
<span class="c1"># TODO(qinyeli, BEAM-646): Remove runner interception of apply.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">pvalueish</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveRunner.run_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.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="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s1">&#39;_desired_cache_labels&#39;</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_desired_cache_labels</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="c1"># Invoke a round trip through the runner API. This makes sure the Pipeline</span>
<span class="c1"># proto is stable.</span>
<span class="n">pipeline</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">Pipeline</span><span class="o">.</span><span class="n">from_runner_api</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">pipeline</span><span class="o">.</span><span class="n">runner</span><span class="p">,</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">_options</span><span class="p">)</span>
<span class="c1"># Snapshot the pipeline in a portable proto before mutating it.</span>
<span class="n">pipeline_proto</span><span class="p">,</span> <span class="n">original_context</span> <span class="o">=</span> <span class="n">pipeline</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">(</span>
<span class="n">return_context</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">pcolls_to_pcoll_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcolls_to_pcoll_id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">original_context</span><span class="p">)</span>
<span class="n">analyzer</span> <span class="o">=</span> <span class="n">pipeline_analyzer</span><span class="o">.</span><span class="n">PipelineAnalyzer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="p">,</span>
<span class="n">pipeline_proto</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">_options</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_desired_cache_labels</span><span class="p">)</span>
<span class="c1"># Should be only accessed for debugging purpose.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_analyzer</span> <span class="o">=</span> <span class="n">analyzer</span>
<span class="n">pipeline_to_execute</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">Pipeline</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span>
<span class="n">analyzer</span><span class="o">.</span><span class="n">pipeline_proto_to_execute</span><span class="p">(),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">_options</span><span class="p">)</span>
<span class="n">display</span> <span class="o">=</span> <span class="n">display_manager</span><span class="o">.</span><span class="n">DisplayManager</span><span class="p">(</span>
<span class="n">pipeline_proto</span><span class="o">=</span><span class="n">pipeline_proto</span><span class="p">,</span>
<span class="n">pipeline_analyzer</span><span class="o">=</span><span class="n">analyzer</span><span class="p">,</span>
<span class="n">cache_manager</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="p">,</span>
<span class="n">pipeline_graph_renderer</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_renderer</span><span class="p">)</span>
<span class="n">display</span><span class="o">.</span><span class="n">start_periodic_update</span><span class="p">()</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">run</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="n">display</span><span class="o">.</span><span class="n">stop_periodic_update</span><span class="p">()</span>
<span class="k">return</span> <span class="n">PipelineResult</span><span class="p">(</span><span class="n">result</span><span class="p">,</span> <span class="bp">self</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_analyzer</span><span class="o">.</span><span class="n">pipeline_info</span><span class="p">(),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="p">,</span> <span class="n">pcolls_to_pcoll_id</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_pcolls_to_pcoll_id</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">original_context</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a dict mapping PCollections string to PCollection IDs.</span>
<span class="sd"> Using a PipelineVisitor to iterate over every node in the pipeline,</span>
<span class="sd"> records the mapping from PCollections to PCollections IDs. This mapping</span>
<span class="sd"> will be used to query cached PCollections.</span>
<span class="sd"> Args:</span>
<span class="sd"> pipeline: (pipeline.Pipeline)</span>
<span class="sd"> original_context: (pipeline_context.PipelineContext)</span>
<span class="sd"> Returns:</span>
<span class="sd"> (dict from str to str) a dict mapping str(pcoll) to pcoll_id.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">pcolls_to_pcoll_id</span> <span class="o">=</span> <span class="p">{}</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PipelineVisitor</span> <span class="c1"># pylint: disable=import-error</span>
<span class="k">class</span> <span class="nc">PCollVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> <span class="c1"># pylint: disable=used-before-assignment</span>
<span class="sd">&quot;&quot;&quot;&quot;A visitor that records input and output values to be replaced.</span>
<span class="sd"> Input and output values that should be updated are recorded in maps</span>
<span class="sd"> input_replacements and output_replacements respectively.</span>
<span class="sd"> We cannot update input and output values while visiting since that</span>
<span class="sd"> results in validation errors.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">enter_composite_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="bp">self</span><span class="o">.</span><span class="n">visit_transform</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</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">for</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="n">pcolls_to_pcoll_id</span><span class="p">[</span><span class="nb">str</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)]</span> <span class="o">=</span> <span class="n">original_context</span><span class="o">.</span><span class="n">pcollections</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span>
<span class="n">pcoll</span><span class="p">)</span>
<span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="n">PCollVisitor</span><span class="p">())</span>
<span class="k">return</span> <span class="n">pcolls_to_pcoll_id</span></div>
<div class="viewcode-block" id="PipelineResult"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.PipelineResult">[docs]</a><span class="k">class</span> <span class="nc">PipelineResult</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">runners</span><span class="o">.</span><span class="n">runner</span><span class="o">.</span><span class="n">PipelineResult</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Provides access to information about a pipeline.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">underlying_result</span><span class="p">,</span> <span class="n">runner</span><span class="p">,</span> <span class="n">pipeline_info</span><span class="p">,</span> <span class="n">cache_manager</span><span class="p">,</span>
<span class="n">pcolls_to_pcoll_id</span><span class="p">):</span>
<span class="nb">super</span><span class="p">(</span><span class="n">PipelineResult</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="n">underlying_result</span><span class="o">.</span><span class="n">state</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="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span> <span class="o">=</span> <span class="n">pipeline_info</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span> <span class="o">=</span> <span class="n">cache_manager</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pcolls_to_pcoll_id</span> <span class="o">=</span> <span class="n">pcolls_to_pcoll_id</span>
<span class="k">def</span> <span class="nf">_cache_label</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="n">pcoll_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcolls_to_pcoll_id</span><span class="p">[</span><span class="nb">str</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)]</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span><span class="o">.</span><span class="n">cache_label</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<div class="viewcode-block" id="PipelineResult.wait_until_finish"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_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="c1"># PipelineResult is not constructed until pipeline execution is finished.</span>
<span class="k">return</span></div>
<div class="viewcode-block" id="PipelineResult.get"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.PipelineResult.get">[docs]</a> <span class="k">def</span> <span class="nf">get</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="n">cache_label</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_label</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="s1">&#39;full&#39;</span><span class="p">,</span> <span class="n">cache_label</span><span class="p">):</span>
<span class="n">pcoll_list</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="s1">&#39;full&#39;</span><span class="p">,</span> <span class="n">cache_label</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pcoll_list</span>
<span class="k">else</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">_desired_cache_labels</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">cache_label</span><span class="p">)</span> <span class="c1"># pylint: disable=protected-access</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;PCollection not available, please run the pipeline.&#39;</span><span class="p">)</span></div>
<div class="viewcode-block" id="PipelineResult.sample"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.PipelineResult.sample">[docs]</a> <span class="k">def</span> <span class="nf">sample</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="n">cache_label</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_label</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="s1">&#39;sample&#39;</span><span class="p">,</span> <span class="n">cache_label</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="s1">&#39;sample&#39;</span><span class="p">,</span> <span class="n">cache_label</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">_runner</span><span class="o">.</span><span class="n">_desired_cache_labels</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">cache_label</span><span class="p">)</span> <span class="c1"># pylint: disable=protected-access</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;PCollection not available, please run the pipeline.&#39;</span><span class="p">)</span></div></div>
</pre></div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright .
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
var DOCUMENTATION_OPTIONS = {
URL_ROOT:'../../../../',
VERSION:'',
COLLAPSE_INDEX:false,
FILE_SUFFIX:'.html',
HAS_SOURCE: true,
SOURCELINK_SUFFIX: '.txt'
};
</script>
<script type="text/javascript" src="../../../../_static/jquery.js"></script>
<script type="text/javascript" src="../../../../_static/underscore.js"></script>
<script type="text/javascript" src="../../../../_static/doctools.js"></script>
<script type="text/javascript" src="../../../../_static/js/theme.js"></script>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.StickyNav.enable();
});
</script>
</body>
</html>