| |
| |
| <!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.pipeline_instrument — Apache Beam 2.47.0 documentation</title> |
| |
| |
| |
| |
| |
| |
| |
| |
| <script type="text/javascript" src="../../../../_static/js/modernizr.min.js"></script> |
| |
| |
| <script type="text/javascript" id="documentation_options" data-url_root="../../../../" src="../../../../_static/documentation_options.js"></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/language_data.js"></script> |
| <script async="async" type="text/javascript" src="https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.5/latest.js?config=TeX-AMS-MML_HTMLorMML"></script> |
| |
| <script type="text/javascript" src="../../../../_static/js/theme.js"></script> |
| |
| |
| |
| |
| <link rel="stylesheet" href="../../../../_static/css/theme.css" type="text/css" /> |
| <link rel="stylesheet" href="../../../../_static/pygments.css" type="text/css" /> |
| <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 class="version"> |
| 2.47.0 |
| </div> |
| |
| |
| |
| |
| <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.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="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.interactive.pipeline_instrument</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.pipeline_instrument</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">"""Module to instrument interactivity to the given pipeline.</span> |
| |
| <span class="sd">For internal use only; no backwards-compatibility guarantees.</span> |
| <span class="sd">This module accesses current interactive environment and analyzes given pipeline</span> |
| <span class="sd">to transform original pipeline into a one-shot pipeline with interactivity.</span> |
| <span class="sd">"""</span> |
| <span class="c1"># pytype: skip-file</span> |
| |
| <span class="kn">import</span> <span class="nn">logging</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Dict</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.pipeline</span> <span class="kn">import</span> <span class="n">PipelineVisitor</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">beam_runner_api_pb2</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="kn">import</span> <span class="n">interactive_environment</span> <span class="k">as</span> <span class="n">ie</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="kn">import</span> <span class="n">pipeline_fragment</span> <span class="k">as</span> <span class="n">pf</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="kn">import</span> <span class="n">background_caching_job</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="kn">import</span> <span class="n">utils</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.caching.cacheable</span> <span class="kn">import</span> <span class="n">Cacheable</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.caching.cacheable</span> <span class="kn">import</span> <span class="n">CacheKey</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.caching.reify</span> <span class="kn">import</span> <span class="n">WRITE_CACHE</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.caching.reify</span> <span class="kn">import</span> <span class="n">reify_to_cache</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.caching.reify</span> <span class="kn">import</span> <span class="n">unreify_from_cache</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.testing</span> <span class="kn">import</span> <span class="n">test_stream</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> |
| |
| |
| <div class="viewcode-block" id="PipelineInstrument"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument">[docs]</a><span class="k">class</span> <span class="nc">PipelineInstrument</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A pipeline instrument for pipeline to be executed by interactive runner.</span> |
| |
| <span class="sd"> This module should never depend on underlying runner that interactive runner</span> |
| <span class="sd"> delegates. It instruments the original instance of pipeline directly by</span> |
| <span class="sd"> appending or replacing transforms with help of cache. It provides</span> |
| <span class="sd"> interfaces to recover states of original pipeline. It's the interactive</span> |
| <span class="sd"> runner's responsibility to coordinate supported underlying runners to run</span> |
| <span class="sd"> the pipeline instrumented and recover the original pipeline states if needed.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">options</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span> <span class="o">=</span> <span class="n">pipeline</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span> <span class="o">=</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">user_pipeline</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="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span> <span class="o">=</span> <span class="n">pipeline</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span> <span class="o">=</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span><span class="p">,</span> <span class="n">create_if_absent</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="c1"># Check if the user defined pipeline contains any source to cache.</span> |
| <span class="c1"># If so, during the check, the cache manager is converted into a</span> |
| <span class="c1"># streaming cache manager, thus re-assign.</span> |
| <span class="k">if</span> <span class="n">background_caching_job</span><span class="o">.</span><span class="n">has_source_to_cache</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</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">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_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">options</span><span class="p">)</span> |
| <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">add_derived_pipeline</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_pipeline</span><span class="p">)</span> |
| |
| <span class="c1"># Snapshot of original pipeline information.</span> |
| <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_original_pipeline_proto</span><span class="p">,</span> |
| <span class="n">context</span><span class="p">)</span> <span class="o">=</span> <span class="bp">self</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="c1"># All compute-once-against-original-pipeline fields.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_unbounded_sources</span> <span class="o">=</span> <span class="n">utils</span><span class="o">.</span><span class="n">unbounded_sources</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_pipeline</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pcoll_to_pcoll_id</span> <span class="o">=</span> <span class="n">pcoll_to_pcoll_id</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">,</span> <span class="n">context</span><span class="p">)</span> |
| |
| <span class="c1"># A Dict[str, Cacheable] from a PCollection id to a Cacheable that belongs</span> |
| <span class="c1"># to the analyzed pipeline.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cacheables</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">find_cacheables</span><span class="p">()</span> |
| |
| <span class="c1"># A dict from cache key to PCollection that is read from cache.</span> |
| <span class="c1"># If exists, caller should reuse the PCollection read. If not, caller</span> |
| <span class="c1"># should create new transform and track the PCollection read from cache.</span> |
| <span class="c1"># (Dict[str, AppliedPTransform]).</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cached_pcoll_read</span> <span class="o">=</span> <span class="p">{}</span> |
| |
| <span class="c1"># A dict from PCollections in the runner pipeline instance to their</span> |
| <span class="c1"># corresponding PCollections in the user pipeline instance. Populated</span> |
| <span class="c1"># after preprocess().</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_runner_pcoll_to_user_pcoll</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pruned_pipeline_proto</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="c1"># Refers target pcolls output by instrumented write cache transforms, used</span> |
| <span class="c1"># by pruning logic as supplemental targets to build pipeline fragment up</span> |
| <span class="c1"># from.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_extended_targets</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| |
| <span class="c1"># Refers pcolls used as inputs but got replaced by outputs of read cache</span> |
| <span class="c1"># transforms instrumented, used by pruning logic as targets no longer need</span> |
| <span class="c1"># to be produced during pipeline runs.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ignored_targets</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| |
| <span class="c1"># Set of PCollections that are written to cache.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">cached_pcolls</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| |
| <div class="viewcode-block" id="PipelineInstrument.instrumented_pipeline_proto"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument.instrumented_pipeline_proto">[docs]</a> <span class="k">def</span> <span class="nf">instrumented_pipeline_proto</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Always returns a new instance of portable instrumented proto."""</span> |
| <span class="n">targets</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_runner_pcoll_to_user_pcoll</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| <span class="n">targets</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_extended_targets</span><span class="p">)</span> |
| <span class="n">targets</span> <span class="o">=</span> <span class="n">targets</span><span class="o">.</span><span class="n">difference</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_ignored_targets</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">targets</span><span class="p">)</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="c1"># Prunes upstream transforms that don't contribute to the targets the</span> |
| <span class="c1"># instrumented pipeline run cares.</span> |
| <span class="k">return</span> <span class="n">pf</span><span class="o">.</span><span class="n">PipelineFragment</span><span class="p">(</span> |
| <span class="nb">list</span><span class="p">(</span><span class="n">targets</span><span class="p">))</span><span class="o">.</span><span class="n">deduce_fragment</span><span class="p">()</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">()</span> |
| <span class="k">return</span> <span class="bp">self</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></div> |
| |
| <span class="k">def</span> <span class="nf">_required_components</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">required_transforms_ids</span><span class="p">,</span> |
| <span class="n">visited</span><span class="p">,</span> |
| <span class="n">follow_outputs</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">follow_inputs</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns the components and subcomponents of the given transforms.</span> |
| |
| <span class="sd"> This method returns required components such as transforms and PCollections</span> |
| <span class="sd"> related to the given transforms and to all of their subtransforms. This</span> |
| <span class="sd"> method accomplishes this recursively.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">required_transforms_ids</span><span class="p">:</span> |
| <span class="k">return</span> <span class="p">({},</span> <span class="p">{})</span> |
| |
| <span class="n">transforms</span> <span class="o">=</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="n">pcollections</span> <span class="o">=</span> <span class="n">pipeline_proto</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">pcollections</span> |
| |
| <span class="c1"># Cache the transforms that will be copied into the new pipeline proto.</span> |
| <span class="n">required_transforms</span> <span class="o">=</span> <span class="p">{</span><span class="n">k</span><span class="p">:</span> <span class="n">transforms</span><span class="p">[</span><span class="n">k</span><span class="p">]</span> <span class="k">for</span> <span class="n">k</span> <span class="ow">in</span> <span class="n">required_transforms_ids</span><span class="p">}</span> |
| |
| <span class="c1"># Cache all the output PCollections of the transforms.</span> |
| <span class="n">pcollection_ids</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">pc</span> <span class="k">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="n">required_transforms</span><span class="o">.</span><span class="n">values</span><span class="p">()</span> <span class="k">for</span> <span class="n">pc</span> <span class="ow">in</span> <span class="n">t</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="p">]</span> |
| <span class="n">required_pcollections</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">pc_id</span><span class="p">:</span> <span class="n">pcollections</span><span class="p">[</span><span class="n">pc_id</span><span class="p">]</span> |
| <span class="k">for</span> <span class="n">pc_id</span> <span class="ow">in</span> <span class="n">pcollection_ids</span> |
| <span class="p">}</span> |
| |
| <span class="n">subtransforms</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="n">subpcollections</span> <span class="o">=</span> <span class="p">{}</span> |
| |
| <span class="c1"># Recursively go through all the subtransforms and add their components.</span> |
| <span class="k">for</span> <span class="n">transform_id</span><span class="p">,</span> <span class="n">transform</span> <span class="ow">in</span> <span class="n">required_transforms</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">transform_id</span> <span class="ow">in</span> <span class="n">pipeline_proto</span><span class="o">.</span><span class="n">root_transform_ids</span><span class="p">:</span> |
| <span class="k">continue</span> |
| <span class="p">(</span><span class="n">t</span><span class="p">,</span> <span class="n">pc</span><span class="p">)</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_required_components</span><span class="p">(</span> |
| <span class="n">pipeline_proto</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">subtransforms</span><span class="p">,</span> |
| <span class="n">visited</span><span class="p">,</span> |
| <span class="n">follow_outputs</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">follow_inputs</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span> |
| <span class="n">subtransforms</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">t</span><span class="p">)</span> |
| <span class="n">subpcollections</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">pc</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">follow_outputs</span><span class="p">:</span> |
| <span class="n">outputs</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">pc_id</span> <span class="k">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="n">required_transforms</span><span class="o">.</span><span class="n">values</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">pc_id</span> <span class="ow">in</span> <span class="n">t</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="p">]</span> |
| <span class="n">visited_copy</span> <span class="o">=</span> <span class="n">visited</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="n">consuming_transforms</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">t_id</span><span class="p">:</span> <span class="n">t</span> |
| <span class="k">for</span> <span class="n">t_id</span><span class="p">,</span> |
| <span class="n">t</span> <span class="ow">in</span> <span class="n">transforms</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="k">if</span> <span class="nb">set</span><span class="p">(</span><span class="n">outputs</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">t</span><span class="o">.</span><span class="n">inputs</span><span class="o">.</span><span class="n">values</span><span class="p">()))</span> |
| <span class="p">}</span> |
| <span class="n">consuming_transforms</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">consuming_transforms</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| <span class="n">visited</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">consuming_transforms</span><span class="p">)</span> |
| <span class="n">consuming_transforms</span> <span class="o">=</span> <span class="n">consuming_transforms</span> <span class="o">-</span> <span class="n">visited_copy</span> |
| <span class="p">(</span><span class="n">t</span><span class="p">,</span> <span class="n">pc</span><span class="p">)</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_required_components</span><span class="p">(</span> |
| <span class="n">pipeline_proto</span><span class="p">,</span> |
| <span class="nb">list</span><span class="p">(</span><span class="n">consuming_transforms</span><span class="p">),</span> |
| <span class="n">visited</span><span class="p">,</span> |
| <span class="n">follow_outputs</span><span class="p">,</span> |
| <span class="n">follow_inputs</span><span class="p">)</span> |
| <span class="n">subtransforms</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">t</span><span class="p">)</span> |
| <span class="n">subpcollections</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">pc</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">follow_inputs</span><span class="p">:</span> |
| <span class="n">inputs</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">pc_id</span> <span class="k">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="n">required_transforms</span><span class="o">.</span><span class="n">values</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">pc_id</span> <span class="ow">in</span> <span class="n">t</span><span class="o">.</span><span class="n">inputs</span><span class="o">.</span><span class="n">values</span><span class="p">()</span> |
| <span class="p">]</span> |
| <span class="n">producing_transforms</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">t_id</span><span class="p">:</span> <span class="n">t</span> |
| <span class="k">for</span> <span class="n">t_id</span><span class="p">,</span> |
| <span class="n">t</span> <span class="ow">in</span> <span class="n">transforms</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="k">if</span> <span class="nb">set</span><span class="p">(</span><span class="n">inputs</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">t</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="p">}</span> |
| <span class="p">(</span><span class="n">t</span><span class="p">,</span> <span class="n">pc</span><span class="p">)</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_required_components</span><span class="p">(</span> |
| <span class="n">pipeline_proto</span><span class="p">,</span> |
| <span class="nb">list</span><span class="p">(</span><span class="n">producing_transforms</span><span class="o">.</span><span class="n">keys</span><span class="p">()),</span> |
| <span class="n">visited</span><span class="p">,</span> |
| <span class="n">follow_outputs</span><span class="p">,</span> |
| <span class="n">follow_inputs</span><span class="p">)</span> |
| <span class="n">subtransforms</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">t</span><span class="p">)</span> |
| <span class="n">subpcollections</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">pc</span><span class="p">)</span> |
| |
| <span class="c1"># Now we got all the components and their subcomponents, so return the</span> |
| <span class="c1"># complete collection.</span> |
| <span class="n">required_transforms</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">subtransforms</span><span class="p">)</span> |
| <span class="n">required_pcollections</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">subpcollections</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="p">(</span><span class="n">required_transforms</span><span class="p">,</span> <span class="n">required_pcollections</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="PipelineInstrument.prune_subgraph_for"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument.prune_subgraph_for">[docs]</a> <span class="k">def</span> <span class="nf">prune_subgraph_for</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">required_transform_ids</span><span class="p">):</span> |
| <span class="c1"># Create the pipeline_proto to read all the components from. It will later</span> |
| <span class="c1"># create a new pipeline proto from the cut out components.</span> |
| <span class="n">pipeline_proto</span><span class="p">,</span> <span class="n">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="c1"># Get all the root transforms. The caching transforms will be subtransforms</span> |
| <span class="c1"># of one of these roots.</span> |
| <span class="n">roots</span> <span class="o">=</span> <span class="p">[</span><span class="n">root</span> <span class="k">for</span> <span class="n">root</span> <span class="ow">in</span> <span class="n">pipeline_proto</span><span class="o">.</span><span class="n">root_transform_ids</span><span class="p">]</span> |
| |
| <span class="p">(</span><span class="n">t</span><span class="p">,</span> <span class="n">p</span><span class="p">)</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_required_components</span><span class="p">(</span> |
| <span class="n">pipeline_proto</span><span class="p">,</span> |
| <span class="n">roots</span> <span class="o">+</span> <span class="n">required_transform_ids</span><span class="p">,</span> |
| <span class="nb">set</span><span class="p">(),</span> |
| <span class="n">follow_outputs</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">follow_inputs</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">set_proto_map</span><span class="p">(</span><span class="n">proto_map</span><span class="p">,</span> <span class="n">new_value</span><span class="p">):</span> |
| <span class="n">proto_map</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">new_value</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="n">proto_map</span><span class="p">[</span><span class="n">key</span><span class="p">]</span><span class="o">.</span><span class="n">CopyFrom</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> |
| |
| <span class="c1"># Copy the transforms into the new pipeline.</span> |
| <span class="n">pipeline_to_execute</span> <span class="o">=</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">pipeline_to_execute</span><span class="o">.</span><span class="n">root_transform_ids</span><span class="p">[:]</span> <span class="o">=</span> <span class="n">roots</span> |
| <span class="n">set_proto_map</span><span class="p">(</span><span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span><span class="p">,</span> <span class="n">t</span><span class="p">)</span> |
| <span class="n">set_proto_map</span><span class="p">(</span><span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">pcollections</span><span class="p">,</span> <span class="n">p</span><span class="p">)</span> |
| <span class="n">set_proto_map</span><span class="p">(</span> |
| <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">coders</span><span class="p">,</span> <span class="n">context</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">()</span><span class="o">.</span><span class="n">coders</span><span class="p">)</span> |
| <span class="n">set_proto_map</span><span class="p">(</span> |
| <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">windowing_strategies</span><span class="p">,</span> |
| <span class="n">context</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">()</span><span class="o">.</span><span class="n">windowing_strategies</span><span class="p">)</span> |
| |
| <span class="c1"># Cut out all subtransforms in the root that aren't the required transforms.</span> |
| <span class="k">for</span> <span class="n">root_id</span> <span class="ow">in</span> <span class="n">roots</span><span class="p">:</span> |
| <span class="n">root</span> <span class="o">=</span> <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span><span class="p">[</span><span class="n">root_id</span><span class="p">]</span> |
| <span class="n">root</span><span class="o">.</span><span class="n">subtransforms</span><span class="p">[:]</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">transform_id</span> <span class="k">for</span> <span class="n">transform_id</span> <span class="ow">in</span> <span class="n">root</span><span class="o">.</span><span class="n">subtransforms</span> |
| <span class="k">if</span> <span class="n">transform_id</span> <span class="ow">in</span> <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span> |
| <span class="p">]</span> |
| |
| <span class="k">return</span> <span class="n">pipeline_to_execute</span></div> |
| |
| <div class="viewcode-block" id="PipelineInstrument.background_caching_pipeline_proto"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument.background_caching_pipeline_proto">[docs]</a> <span class="k">def</span> <span class="nf">background_caching_pipeline_proto</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns the background caching pipeline.</span> |
| |
| <span class="sd"> This method creates a background caching pipeline by: adding writes to cache</span> |
| <span class="sd"> from each unbounded source (done in the instrument method), and cutting out</span> |
| <span class="sd"> all components (transform, PCollections, coders, windowing strategies) that</span> |
| <span class="sd"> are not the unbounded sources or writes to cache (or subtransforms thereof).</span> |
| <span class="sd"> """</span> |
| <span class="c1"># Create the pipeline_proto to read all the components from. It will later</span> |
| <span class="c1"># create a new pipeline proto from the cut out components.</span> |
| <span class="n">pipeline_proto</span><span class="p">,</span> <span class="n">context</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_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="c1"># Get all the sources we want to cache.</span> |
| <span class="n">sources</span> <span class="o">=</span> <span class="n">utils</span><span class="o">.</span><span class="n">unbounded_sources</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_pipeline</span><span class="p">)</span> |
| |
| <span class="c1"># Get all the root transforms. The caching transforms will be subtransforms</span> |
| <span class="c1"># of one of these roots.</span> |
| <span class="n">roots</span> <span class="o">=</span> <span class="p">[</span><span class="n">root</span> <span class="k">for</span> <span class="n">root</span> <span class="ow">in</span> <span class="n">pipeline_proto</span><span class="o">.</span><span class="n">root_transform_ids</span><span class="p">]</span> |
| |
| <span class="c1"># Get the transform IDs of the caching transforms. These caching operations</span> |
| <span class="c1"># are added to the _background_caching_pipeline in the instrument() method.</span> |
| <span class="c1"># It's added there so that multiple calls to this method won't add multiple</span> |
| <span class="c1"># caching operations (idempotent).</span> |
| <span class="n">transforms</span> <span class="o">=</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="n">caching_transform_ids</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">t_id</span> <span class="k">for</span> <span class="n">root</span> <span class="ow">in</span> <span class="n">roots</span> <span class="k">for</span> <span class="n">t_id</span> <span class="ow">in</span> <span class="n">transforms</span><span class="p">[</span><span class="n">root</span><span class="p">]</span><span class="o">.</span><span class="n">subtransforms</span> |
| <span class="k">if</span> <span class="n">WRITE_CACHE</span> <span class="ow">in</span> <span class="n">t_id</span> |
| <span class="p">]</span> |
| |
| <span class="c1"># Get the IDs of the unbounded sources.</span> |
| <span class="n">required_transform_labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">src</span><span class="o">.</span><span class="n">full_label</span> <span class="k">for</span> <span class="n">src</span> <span class="ow">in</span> <span class="n">sources</span><span class="p">]</span> |
| <span class="n">unbounded_source_ids</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">k</span> <span class="k">for</span> <span class="n">k</span><span class="p">,</span> |
| <span class="n">v</span> <span class="ow">in</span> <span class="n">transforms</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> <span class="k">if</span> <span class="n">v</span><span class="o">.</span><span class="n">unique_name</span> <span class="ow">in</span> <span class="n">required_transform_labels</span> |
| <span class="p">]</span> |
| |
| <span class="c1"># The required transforms are the transforms that we want to cut out of</span> |
| <span class="c1"># the pipeline_proto and insert into a new pipeline to return.</span> |
| <span class="n">required_transform_ids</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">roots</span> <span class="o">+</span> <span class="n">caching_transform_ids</span> <span class="o">+</span> <span class="n">unbounded_source_ids</span><span class="p">)</span> |
| <span class="p">(</span><span class="n">t</span><span class="p">,</span> <span class="n">p</span><span class="p">)</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_required_components</span><span class="p">(</span> |
| <span class="n">pipeline_proto</span><span class="p">,</span> <span class="n">required_transform_ids</span><span class="p">,</span> <span class="nb">set</span><span class="p">())</span> |
| |
| <span class="k">def</span> <span class="nf">set_proto_map</span><span class="p">(</span><span class="n">proto_map</span><span class="p">,</span> <span class="n">new_value</span><span class="p">):</span> |
| <span class="n">proto_map</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">new_value</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="n">proto_map</span><span class="p">[</span><span class="n">key</span><span class="p">]</span><span class="o">.</span><span class="n">CopyFrom</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> |
| |
| <span class="c1"># Copy the transforms into the new pipeline.</span> |
| <span class="n">pipeline_to_execute</span> <span class="o">=</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">pipeline_to_execute</span><span class="o">.</span><span class="n">root_transform_ids</span><span class="p">[:]</span> <span class="o">=</span> <span class="n">roots</span> |
| <span class="n">set_proto_map</span><span class="p">(</span><span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span><span class="p">,</span> <span class="n">t</span><span class="p">)</span> |
| <span class="n">set_proto_map</span><span class="p">(</span><span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">pcollections</span><span class="p">,</span> <span class="n">p</span><span class="p">)</span> |
| <span class="n">set_proto_map</span><span class="p">(</span> |
| <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">coders</span><span class="p">,</span> <span class="n">context</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">()</span><span class="o">.</span><span class="n">coders</span><span class="p">)</span> |
| <span class="n">set_proto_map</span><span class="p">(</span> |
| <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">windowing_strategies</span><span class="p">,</span> |
| <span class="n">context</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">()</span><span class="o">.</span><span class="n">windowing_strategies</span><span class="p">)</span> |
| |
| <span class="c1"># Cut out all subtransforms in the root that aren't the required transforms.</span> |
| <span class="k">for</span> <span class="n">root_id</span> <span class="ow">in</span> <span class="n">roots</span><span class="p">:</span> |
| <span class="n">root</span> <span class="o">=</span> <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span><span class="p">[</span><span class="n">root_id</span><span class="p">]</span> |
| <span class="n">root</span><span class="o">.</span><span class="n">subtransforms</span><span class="p">[:]</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">transform_id</span> <span class="k">for</span> <span class="n">transform_id</span> <span class="ow">in</span> <span class="n">root</span><span class="o">.</span><span class="n">subtransforms</span> |
| <span class="k">if</span> <span class="n">transform_id</span> <span class="ow">in</span> <span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span> |
| <span class="p">]</span> |
| |
| <span class="k">return</span> <span class="n">pipeline_to_execute</span></div> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">cacheables</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-></span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Cacheable</span><span class="p">]:</span> |
| <span class="w"> </span><span class="sd">"""Returns the Cacheables by PCollection ids.</span> |
| |
| <span class="sd"> If you're already working with user defined pipelines and PCollections,</span> |
| <span class="sd"> do not build a PipelineInstrument just to get the cacheables. Instead,</span> |
| <span class="sd"> use apache_beam.runners.interactive.utils.cacheables.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cacheables</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">has_unbounded_sources</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns whether the pipeline has any recordable sources.</span> |
| <span class="sd"> """</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">_unbounded_sources</span><span class="p">)</span> <span class="o">></span> <span class="mi">0</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">original_pipeline_proto</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns a snapshot of the pipeline proto before instrumentation."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_original_pipeline_proto</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">user_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns a reference to the pipeline instance defined by the user. If a</span> |
| <span class="sd"> pipeline has no cacheable PCollection and the user pipeline cannot be</span> |
| <span class="sd"> found, return None indicating there is nothing to be cached in the user</span> |
| <span class="sd"> pipeline.</span> |
| |
| <span class="sd"> The pipeline given for instrumenting and mutated in this class is not</span> |
| <span class="sd"> necessarily the pipeline instance defined by the user. From the watched</span> |
| <span class="sd"> scopes, this class figures out what the user pipeline instance is.</span> |
| <span class="sd"> This metadata can be used for tracking pipeline results.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">runner_pcoll_to_user_pcoll</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns cacheable PCollections correlated from instances in the runner</span> |
| <span class="sd"> pipeline to instances in the user pipeline."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_runner_pcoll_to_user_pcoll</span> |
| |
| <div class="viewcode-block" id="PipelineInstrument.find_cacheables"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument.find_cacheables">[docs]</a> <span class="k">def</span> <span class="nf">find_cacheables</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-></span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Cacheable</span><span class="p">]:</span> |
| <span class="w"> </span><span class="sd">"""Finds PCollections that need to be cached for analyzed pipeline.</span> |
| |
| <span class="sd"> There might be multiple pipelines defined and watched, this will only find</span> |
| <span class="sd"> cacheables belong to the analyzed pipeline.</span> |
| <span class="sd"> """</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="n">cacheables</span> <span class="o">=</span> <span class="n">utils</span><span class="o">.</span><span class="n">cacheables</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">cacheable</span> <span class="ow">in</span> <span class="n">cacheables</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">cacheable</span><span class="o">.</span><span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span> <span class="ow">is</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span><span class="p">:</span> |
| <span class="c1"># Ignore all cacheables from other pipelines.</span> |
| <span class="k">continue</span> |
| <span class="n">pcoll_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">pcoll_id</span><span class="p">(</span><span class="n">cacheable</span><span class="o">.</span><span class="n">pcoll</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">pcoll_id</span><span class="p">:</span> |
| <span class="n">_LOGGER</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span> |
| <span class="s1">'Unable to retrieve PCollection id for </span><span class="si">%s</span><span class="s1">. Ignored.'</span><span class="p">,</span> |
| <span class="n">cacheable</span><span class="o">.</span><span class="n">pcoll</span><span class="p">)</span> |
| <span class="k">continue</span> |
| <span class="n">result</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">pcoll_id</span><span class="p">(</span><span class="n">cacheable</span><span class="o">.</span><span class="n">pcoll</span><span class="p">)]</span> <span class="o">=</span> <span class="n">cacheable</span> |
| <span class="k">return</span> <span class="n">result</span></div> |
| |
| <div class="viewcode-block" id="PipelineInstrument.instrument"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument.instrument">[docs]</a> <span class="k">def</span> <span class="nf">instrument</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Instruments original pipeline with cache.</span> |
| |
| <span class="sd"> For cacheable output PCollection, if cache for the key doesn't exist, do</span> |
| <span class="sd"> _write_cache(); for cacheable input PCollection, if cache for the key</span> |
| <span class="sd"> exists, do _read_cache(). No instrument in any other situation.</span> |
| |
| <span class="sd"> Modifies:</span> |
| <span class="sd"> self._pipeline</span> |
| <span class="sd"> """</span> |
| <span class="n">cacheable_inputs</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| <span class="n">all_inputs</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| <span class="n">all_outputs</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| <span class="n">unbounded_source_pcolls</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| |
| <span class="k">class</span> <span class="nc">InstrumentVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Visitor utilizes cache to instrument the pipeline."""</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pin</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span> <span class="o">=</span> <span class="n">pin</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">if</span> <span class="nb">isinstance</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="nb">tuple</span><span class="p">(</span><span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">recordable_sources</span><span class="p">)):</span> |
| <span class="n">unbounded_source_pcolls</span><span class="o">.</span><span class="n">update</span><span class="p">(</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">cacheable_inputs</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cacheable_inputs</span><span class="p">(</span><span class="n">transform_node</span><span class="p">))</span> |
| <span class="n">ins</span><span class="p">,</span> <span class="n">outs</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_all_inputs_outputs</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| <span class="n">all_inputs</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">ins</span><span class="p">)</span> |
| <span class="n">all_outputs</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">outs</span><span class="p">)</span> |
| |
| <span class="n">v</span> <span class="o">=</span> <span class="n">InstrumentVisitor</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">_pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> |
| <span class="c1"># Every output PCollection that is never used as an input PCollection is</span> |
| <span class="c1"># considered as a side effect of the pipeline run and should be included.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_extended_targets</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">all_outputs</span><span class="o">.</span><span class="n">difference</span><span class="p">(</span><span class="n">all_inputs</span><span class="p">))</span> |
| <span class="c1"># Add the unbounded source PCollections to the cacheable inputs. This allows</span> |
| <span class="c1"># for the caching of unbounded sources without a variable reference.</span> |
| <span class="n">cacheable_inputs</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">unbounded_source_pcolls</span><span class="p">)</span> |
| |
| <span class="c1"># Create ReadCache transforms.</span> |
| <span class="k">for</span> <span class="n">cacheable_input</span> <span class="ow">in</span> <span class="n">cacheable_inputs</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_read_cache</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">,</span> |
| <span class="n">cacheable_input</span><span class="p">,</span> |
| <span class="n">cacheable_input</span> <span class="ow">in</span> <span class="n">unbounded_source_pcolls</span><span class="p">)</span> |
| <span class="c1"># Replace/wire inputs w/ cached PCollections from ReadCache transforms.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_replace_with_cached_inputs</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">)</span> |
| |
| <span class="c1"># Write cache for all cacheables.</span> |
| <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">cacheable</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cacheables</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_write_cache</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">,</span> <span class="n">cacheable</span><span class="o">.</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">ignore_unbounded_reads</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| |
| <span class="c1"># Instrument the background caching pipeline if we can.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">has_unbounded_sources</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">source</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_unbounded_sources</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_write_cache</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_pipeline</span><span class="p">,</span> |
| <span class="n">source</span><span class="o">.</span><span class="n">outputs</span><span class="p">[</span><span class="kc">None</span><span class="p">],</span> |
| <span class="n">output_as_extended_target</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">is_capture</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| |
| <span class="k">class</span> <span class="nc">TestStreamVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">test_stream</span> <span class="o">=</span> <span class="kc">None</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">if</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">test_stream</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> |
| <span class="nb">isinstance</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="n">test_stream</span><span class="o">.</span><span class="n">TestStream</span><span class="p">)):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">test_stream</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span> |
| |
| <span class="n">v</span> <span class="o">=</span> <span class="n">TestStreamVisitor</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> |
| <span class="n">pipeline_proto</span> <span class="o">=</span> <span class="bp">self</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">False</span><span class="p">)</span> |
| <span class="n">test_stream_id</span> <span class="o">=</span> <span class="s1">''</span> |
| <span class="k">for</span> <span class="n">t_id</span><span class="p">,</span> <span class="n">t</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">items</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">t</span><span class="o">.</span><span class="n">unique_name</span> <span class="o">==</span> <span class="n">v</span><span class="o">.</span><span class="n">test_stream</span><span class="p">:</span> |
| <span class="n">test_stream_id</span> <span class="o">=</span> <span class="n">t_id</span> |
| <span class="k">break</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pruned_pipeline_proto</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">prune_subgraph_for</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">,</span> <span class="p">[</span><span class="n">test_stream_id</span><span class="p">])</span> |
| <span class="n">pruned_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">from_runner_api</span><span class="p">(</span> |
| <span class="n">proto</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_pruned_pipeline_proto</span><span class="p">,</span> |
| <span class="n">runner</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="o">.</span><span class="n">runner</span><span class="p">,</span> |
| <span class="n">options</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="o">.</span><span class="n">_options</span><span class="p">)</span> |
| <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">add_derived_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">,</span> <span class="n">pruned_pipeline</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span> <span class="o">=</span> <span class="n">pruned_pipeline</span></div> |
| |
| <div class="viewcode-block" id="PipelineInstrument.preprocess"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument.preprocess">[docs]</a> <span class="k">def</span> <span class="nf">preprocess</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Pre-processes the pipeline.</span> |
| |
| <span class="sd"> Since the pipeline instance in the class might not be the same instance</span> |
| <span class="sd"> defined in the user code, the pre-process will figure out the relationship</span> |
| <span class="sd"> of cacheable PCollections between these 2 instances by replacing 'pcoll'</span> |
| <span class="sd"> fields in the cacheable dictionary with ones from the running instance.</span> |
| <span class="sd"> """</span> |
| <span class="k">class</span> <span class="nc">PreprocessVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pin</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span> <span class="o">=</span> <span class="n">pin</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">in_pcoll</span> <span class="ow">in</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_process</span><span class="p">(</span><span class="n">in_pcoll</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">out_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="bp">self</span><span class="o">.</span><span class="n">_process</span><span class="p">(</span><span class="n">out_pcoll</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">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">_pin</span><span class="o">.</span><span class="n">_pcoll_to_pcoll_id</span><span class="o">.</span><span class="n">get</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="s1">''</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cacheables</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">_pin</span><span class="o">.</span><span class="n">pcoll_id</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span> |
| <span class="n">user_pcoll</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cacheables</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span><span class="o">.</span><span class="n">pcoll</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">pcoll_id</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cacheables</span> <span class="ow">and</span> <span class="n">user_pcoll</span> <span class="o">!=</span> <span class="n">pcoll</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_runner_pcoll_to_user_pcoll</span><span class="p">[</span><span class="n">pcoll</span><span class="p">]</span> <span class="o">=</span> <span class="n">user_pcoll</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cacheables</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span><span class="o">.</span><span class="n">pcoll</span> <span class="o">=</span> <span class="n">pcoll</span> |
| |
| <span class="n">v</span> <span class="o">=</span> <span class="n">PreprocessVisitor</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">_pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="n">v</span><span class="p">)</span></div> |
| |
| <span class="k">def</span> <span class="nf">_write_cache</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">pcoll</span><span class="p">,</span> |
| <span class="n">output_as_extended_target</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">ignore_unbounded_reads</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">is_capture</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Caches a cacheable PCollection.</span> |
| |
| <span class="sd"> For the given PCollection, by appending sub transform part that materialize</span> |
| <span class="sd"> the PCollection through sink into cache implementation. The cache write is</span> |
| <span class="sd"> not immediate. It happens when the runner runs the transformed pipeline</span> |
| <span class="sd"> and thus not usable for this run as intended. This function always writes</span> |
| <span class="sd"> the cache for the given PCollection as long as the PCollection belongs to</span> |
| <span class="sd"> the pipeline being instrumented and the keyed cache is absent.</span> |
| |
| <span class="sd"> Modifies:</span> |
| <span class="sd"> pipeline</span> |
| <span class="sd"> """</span> |
| <span class="c1"># Makes sure the pcoll belongs to the pipeline being instrumented.</span> |
| <span class="k">if</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">pipeline</span><span class="p">:</span> |
| <span class="k">return</span> |
| |
| <span class="c1"># Ignore the unbounded reads from recordable sources as these will be pruned</span> |
| <span class="c1"># out using the PipelineFragment later on.</span> |
| <span class="k">if</span> <span class="n">ignore_unbounded_reads</span><span class="p">:</span> |
| <span class="n">ignore</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="n">producer</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">producer</span> |
| <span class="k">while</span> <span class="n">producer</span><span class="p">:</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">producer</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> |
| <span class="nb">tuple</span><span class="p">(</span><span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">recordable_sources</span><span class="p">)):</span> |
| <span class="n">ignore</span> <span class="o">=</span> <span class="kc">True</span> |
| <span class="k">break</span> |
| <span class="n">producer</span> <span class="o">=</span> <span class="n">producer</span><span class="o">.</span><span class="n">parent</span> |
| <span class="k">if</span> <span class="n">ignore</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ignored_targets</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span> |
| <span class="k">return</span> |
| |
| <span class="c1"># The keyed cache is always valid within this instrumentation.</span> |
| <span class="n">key</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">cache_key</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span> |
| <span class="c1"># Only need to write when the cache with expected key doesn't exist.</span> |
| <span class="k">if</span> <span class="ow">not</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">'full'</span><span class="p">,</span> <span class="n">key</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">cached_pcolls</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">runner_pcoll_to_user_pcoll</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">))</span> |
| <span class="c1"># Read the windowing information and cache it along with the element. This</span> |
| <span class="c1"># caches the arguments to a WindowedValue object because Python has logic</span> |
| <span class="c1"># that detects if a DoFn returns a WindowedValue. When it detecs one, it</span> |
| <span class="c1"># puts the element into the correct window then emits the value to</span> |
| <span class="c1"># downstream transforms.</span> |
| <span class="n">extended_target</span> <span class="o">=</span> <span class="n">reify_to_cache</span><span class="p">(</span> |
| <span class="n">pcoll</span><span class="o">=</span><span class="n">pcoll</span><span class="p">,</span> |
| <span class="n">cache_key</span><span class="o">=</span><span class="n">key</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">is_capture</span><span class="o">=</span><span class="n">is_capture</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">output_as_extended_target</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_extended_targets</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">extended_target</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_read_cache</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">pcoll</span><span class="p">,</span> <span class="n">is_unbounded_source_output</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Reads a cached pvalue.</span> |
| |
| <span class="sd"> A noop will cause the pipeline to execute the transform as</span> |
| <span class="sd"> it is and cache nothing from this transform for next run.</span> |
| |
| <span class="sd"> Modifies:</span> |
| <span class="sd"> pipeline</span> |
| <span class="sd"> """</span> |
| <span class="c1"># Makes sure the pcoll belongs to the pipeline being instrumented.</span> |
| <span class="k">if</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">pipeline</span><span class="p">:</span> |
| <span class="k">return</span> |
| <span class="c1"># The keyed cache is always valid within this instrumentation.</span> |
| <span class="n">key</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">cache_key</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span> |
| <span class="c1"># Can only read from cache when the cache with expected key exists and its</span> |
| <span class="c1"># computation has been completed.</span> |
| <span class="n">is_cached</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">exists</span><span class="p">(</span><span class="s1">'full'</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span> |
| <span class="n">is_computed</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">pcoll</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_runner_pcoll_to_user_pcoll</span> <span class="ow">and</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_runner_pcoll_to_user_pcoll</span><span class="p">[</span><span class="n">pcoll</span><span class="p">]</span> <span class="ow">in</span> |
| <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">computed_pcollections</span><span class="p">)</span> |
| <span class="k">if</span> <span class="p">((</span><span class="n">is_cached</span> <span class="ow">and</span> <span class="n">is_computed</span><span class="p">)</span> <span class="ow">or</span> <span class="n">is_unbounded_source_output</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">key</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cached_pcoll_read</span><span class="p">:</span> |
| <span class="c1"># Mutates the pipeline with cache read transform attached</span> |
| <span class="c1"># to root of the pipeline.</span> |
| |
| <span class="c1"># To put the cached value into the correct window, simply return a</span> |
| <span class="c1"># WindowedValue constructed from the element.</span> |
| <span class="n">pcoll_from_cache</span> <span class="o">=</span> <span class="n">unreify_from_cache</span><span class="p">(</span> |
| <span class="n">pipeline</span><span class="o">=</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">cache_key</span><span class="o">=</span><span class="n">key</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="bp">self</span><span class="o">.</span><span class="n">_cached_pcoll_read</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</span> <span class="n">pcoll_from_cache</span> |
| <span class="c1"># else: NOOP when cache doesn't exist, just compute the original graph.</span> |
| |
| <span class="k">def</span> <span class="nf">_replace_with_cached_inputs</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="w"> </span><span class="sd">"""Replace PCollection inputs in the pipeline with cache if possible.</span> |
| |
| <span class="sd"> For any input PCollection, find out whether there is valid cache. If so,</span> |
| <span class="sd"> replace the input of the AppliedPTransform with output of the</span> |
| <span class="sd"> AppliedPtransform that sources pvalue from the cache. If there is no valid</span> |
| <span class="sd"> cache, noop.</span> |
| <span class="sd"> """</span> |
| |
| <span class="c1"># Find all cached unbounded PCollections.</span> |
| |
| <span class="c1"># If the pipeline has unbounded sources, then we want to force all cache</span> |
| <span class="c1"># reads to go through the TestStream (even if they are bounded sources).</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">has_unbounded_sources</span><span class="p">:</span> |
| |
| <span class="k">class</span> <span class="nc">CacheableUnboundedPCollectionVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pin</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span> <span class="o">=</span> <span class="n">pin</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">unbounded_pcolls</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</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">if</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">output_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">key</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">cache_key</span><span class="p">(</span><span class="n">output_pcoll</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">key</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cached_pcoll_read</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">unbounded_pcolls</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">input_pcoll</span> <span class="ow">in</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">:</span> |
| <span class="n">key</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">cache_key</span><span class="p">(</span><span class="n">input_pcoll</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">key</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cached_pcoll_read</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">unbounded_pcolls</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> |
| |
| <span class="n">v</span> <span class="o">=</span> <span class="n">CacheableUnboundedPCollectionVisitor</span><span class="p">(</span><span class="bp">self</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">v</span><span class="p">)</span> |
| |
| <span class="c1"># The set of keys from the cached unbounded PCollections will be used as</span> |
| <span class="c1"># the output tags for the TestStream. This is to remember what cache-key</span> |
| <span class="c1"># is associated with which PCollection.</span> |
| <span class="n">output_tags</span> <span class="o">=</span> <span class="n">v</span><span class="o">.</span><span class="n">unbounded_pcolls</span> |
| |
| <span class="c1"># Take the PCollections that will be read from the TestStream and insert</span> |
| <span class="c1"># them back into the dictionary of cached PCollections. The next step will</span> |
| <span class="c1"># replace the downstream consumer of the non-cached PCollections with</span> |
| <span class="c1"># these PCollections.</span> |
| <span class="k">if</span> <span class="n">output_tags</span><span class="p">:</span> |
| <span class="n">output_pcolls</span> <span class="o">=</span> <span class="n">pipeline</span> <span class="o">|</span> <span class="n">test_stream</span><span class="o">.</span><span class="n">TestStream</span><span class="p">(</span> |
| <span class="n">output_tags</span><span class="o">=</span><span class="n">output_tags</span><span class="p">,</span> <span class="n">coder</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">_default_pcoder</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">output_pcolls</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cached_pcoll_read</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span> <span class="o">=</span> <span class="n">pcoll</span> |
| |
| <span class="k">class</span> <span class="nc">ReadCacheWireVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Visitor wires cache read as inputs to replace corresponding original</span> |
| <span class="sd"> input PCollections in pipeline.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pin</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Initializes with a PipelineInstrument."""</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span> <span class="o">=</span> <span class="n">pin</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">if</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">:</span> |
| <span class="n">main_inputs</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">main_inputs</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">input_pcoll</span> <span class="ow">in</span> <span class="n">main_inputs</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="n">key</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">cache_key</span><span class="p">(</span><span class="n">input_pcoll</span><span class="p">)</span> |
| |
| <span class="c1"># Replace the input pcollection with the cached pcollection (if it</span> |
| <span class="c1"># has been cached).</span> |
| <span class="k">if</span> <span class="n">key</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cached_pcoll_read</span><span class="p">:</span> |
| <span class="c1"># Ignore this pcoll in the final pruned instrumented pipeline.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_ignored_targets</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">input_pcoll</span><span class="p">)</span> |
| <span class="n">main_inputs</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pin</span><span class="o">.</span><span class="n">_cached_pcoll_read</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> |
| <span class="c1"># Update the transform with its new inputs.</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">main_inputs</span> <span class="o">=</span> <span class="n">main_inputs</span> |
| |
| <span class="n">v</span> <span class="o">=</span> <span class="n">ReadCacheWireVisitor</span><span class="p">(</span><span class="bp">self</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">v</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_cacheable_inputs</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">inputs</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">in_pcoll</span> <span class="ow">in</span> <span class="n">transform</span><span class="o">.</span><span class="n">inputs</span><span class="p">:</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">pcoll_id</span><span class="p">(</span><span class="n">in_pcoll</span><span class="p">)</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cacheables</span><span class="p">:</span> |
| <span class="n">inputs</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">in_pcoll</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">inputs</span> |
| |
| <span class="k">def</span> <span class="nf">_all_inputs_outputs</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">inputs</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| <span class="n">outputs</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">in_pcoll</span> <span class="ow">in</span> <span class="n">transform</span><span class="o">.</span><span class="n">inputs</span><span class="p">:</span> |
| <span class="n">inputs</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">in_pcoll</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">out_pcoll</span> <span class="ow">in</span> <span class="n">transform</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="n">outputs</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">out_pcoll</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">inputs</span><span class="p">,</span> <span class="n">outputs</span> |
| |
| <div class="viewcode-block" id="PipelineInstrument.pcoll_id"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument.pcoll_id">[docs]</a> <span class="k">def</span> <span class="nf">pcoll_id</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="w"> </span><span class="sd">"""Gets the PCollection id of the given pcoll.</span> |
| |
| <span class="sd"> Returns '' if not found.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcoll_to_pcoll_id</span><span class="o">.</span><span class="n">get</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="s1">''</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="PipelineInstrument.cache_key"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.PipelineInstrument.cache_key">[docs]</a> <span class="k">def</span> <span class="nf">cache_key</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="w"> </span><span class="sd">"""Gets the identifier of a cacheable PCollection in cache.</span> |
| |
| <span class="sd"> If the pcoll is not a cacheable, return ''.</span> |
| <span class="sd"> This is only needed in pipeline instrument when the origin of given pcoll</span> |
| <span class="sd"> is unknown (whether it's from the user pipeline or a runner pipeline). If</span> |
| <span class="sd"> a pcoll is from the user pipeline, always use CacheKey.from_pcoll to build</span> |
| <span class="sd"> the key.</span> |
| <span class="sd"> The key is what the pcoll would use as identifier if it's materialized in</span> |
| <span class="sd"> cache. It doesn't mean that there would definitely be such cache already.</span> |
| <span class="sd"> Also, the pcoll can come from the original user defined pipeline object or</span> |
| <span class="sd"> an equivalent pcoll from a transformed copy of the original pipeline.</span> |
| <span class="sd"> """</span> |
| <span class="n">cacheable</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cacheables</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">pcoll_id</span><span class="p">(</span><span class="n">pcoll</span><span class="p">),</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">cacheable</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">cacheable</span><span class="o">.</span><span class="n">pcoll</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">runner_pcoll_to_user_pcoll</span><span class="p">:</span> |
| <span class="n">user_pcoll</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">runner_pcoll_to_user_pcoll</span><span class="p">[</span><span class="n">cacheable</span><span class="o">.</span><span class="n">pcoll</span><span class="p">]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">user_pcoll</span> <span class="o">=</span> <span class="n">cacheable</span><span class="o">.</span><span class="n">pcoll</span> |
| <span class="k">return</span> <span class="n">CacheKey</span><span class="o">.</span><span class="n">from_pcoll</span><span class="p">(</span><span class="n">cacheable</span><span class="o">.</span><span class="n">var</span><span class="p">,</span> <span class="n">user_pcoll</span><span class="p">)</span><span class="o">.</span><span class="n">to_str</span><span class="p">()</span> |
| <span class="k">return</span> <span class="s1">''</span></div></div> |
| |
| |
| <div class="viewcode-block" id="build_pipeline_instrument"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.build_pipeline_instrument">[docs]</a><span class="k">def</span> <span class="nf">build_pipeline_instrument</span><span class="p">(</span><span class="n">pipeline</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="w"> </span><span class="sd">"""Creates PipelineInstrument for a pipeline and its options with cache.</span> |
| |
| <span class="sd"> Throughout the process, the returned PipelineInstrument snapshots the given</span> |
| <span class="sd"> pipeline and then mutates the pipeline. It's invoked by interactive components</span> |
| <span class="sd"> such as the InteractiveRunner and the given pipeline should be implicitly</span> |
| <span class="sd"> created runner pipelines instead of pipeline instances defined by the user.</span> |
| |
| <span class="sd"> This is the shorthand for doing 3 steps: 1) compute once for metadata of the</span> |
| <span class="sd"> given runner pipeline and everything watched from user pipelines; 2) associate</span> |
| <span class="sd"> info between the runner pipeline and its corresponding user pipeline,</span> |
| <span class="sd"> eliminate data from other user pipelines if there are any; 3) mutate the</span> |
| <span class="sd"> runner pipeline to apply interactivity.</span> |
| <span class="sd"> """</span> |
| <span class="n">pi</span> <span class="o">=</span> <span class="n">PipelineInstrument</span><span class="p">(</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span> |
| <span class="n">pi</span><span class="o">.</span><span class="n">preprocess</span><span class="p">()</span> |
| <span class="n">pi</span><span class="o">.</span><span class="n">instrument</span><span class="p">()</span> <span class="c1"># Instruments the pipeline only once.</span> |
| <span class="k">return</span> <span class="n">pi</span></div> |
| |
| |
| <div class="viewcode-block" id="pcoll_to_pcoll_id"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_instrument.html#apache_beam.runners.interactive.pipeline_instrument.pcoll_to_pcoll_id">[docs]</a><span class="k">def</span> <span class="nf">pcoll_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="w"> </span><span class="sd">"""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"> Returns:</span> |
| <span class="sd"> (dict from str to str) a dict mapping str(pcoll) to pcoll_id.</span> |
| <span class="sd"> """</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="w"> </span><span class="sd">""""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"> """</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">pcoll_to_pcoll_id</span> <span class="o">=</span> <span class="p">{}</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="bp">self</span><span class="o">.</span><span class="n">pcoll_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="p">(</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">v</span> <span class="o">=</span> <span class="n">PCollVisitor</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">v</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">v</span><span class="o">.</span><span class="n">pcoll_to_pcoll_id</span></div> |
| </pre></div> |
| |
| </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/rtfd/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"> |
| jQuery(function () { |
| SphinxRtdTheme.Navigation.enable(true); |
| }); |
| </script> |
| |
| |
| |
| |
| |
| |
| </body> |
| </html> |