blob: 4a40bc1bc8a9f93e2ccbe3c8899ac56aa8478f71 [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.runners.interactive.augmented_pipeline &mdash; 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> &raquo;</li>
<li><a href="../../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.runners.interactive.augmented_pipeline</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.augmented_pipeline</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="sd">&quot;&quot;&quot;Module to augment interactive flavor into the given pipeline.</span>
<span class="sd">For internal use only; no backward-compatibility guarantees.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span> <span class="nn">copy</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Dict</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Optional</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Set</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.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">background_caching_job</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.read_cache</span> <span class="kn">import</span> <span class="n">ReadCache</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.caching.write_cache</span> <span class="kn">import</span> <span class="n">WriteCache</span>
<div class="viewcode-block" id="AugmentedPipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.augmented_pipeline.html#apache_beam.runners.interactive.augmented_pipeline.AugmentedPipeline">[docs]</a><span class="k">class</span> <span class="nc">AugmentedPipeline</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A pipeline with augmented interactive flavor that caches intermediate</span>
<span class="sd"> PCollections defined by the user, reads computed PCollections as source and</span>
<span class="sd"> prunes unnecessary pipeline parts for fast computation.</span>
<span class="sd"> &quot;&quot;&quot;</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">user_pipeline</span><span class="p">:</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">,</span>
<span class="n">pcolls</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Set</span><span class="p">[</span><span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Initializes a pipelilne for augmenting interactive flavor.</span>
<span class="sd"> Args:</span>
<span class="sd"> user_pipeline: a beam.Pipeline instance defined by the user.</span>
<span class="sd"> pcolls: cacheable pcolls to be computed/retrieved. If the set is</span>
<span class="sd"> empty, all intermediate pcolls assigned to variables are applicable.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">pcolls</span> <span class="ow">or</span> <span class="nb">all</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span> <span class="ow">is</span> <span class="n">user_pipeline</span> <span class="k">for</span> <span class="n">pcoll</span> <span class="ow">in</span>
<span class="n">pcolls</span><span class="p">),</span> <span class="s1">&#39;All </span><span class="si">%s</span><span class="s1"> need to belong to </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">pcolls</span><span class="p">,</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">user_pipeline</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pcolls</span> <span class="o">=</span> <span class="n">pcolls</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="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="n">_</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_user_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="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">component_id_map</span> <span class="o">=</span> <span class="n">copy</span><span class="o">.</span><span class="n">copy</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">component_id_map</span><span class="p">)</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">cacheables</span><span class="p">()</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">augmented_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">augment</span><span class="p">()</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/20526): Support generating a</span>
<span class="c1"># background recording job that contains unbound source recording transforms</span>
<span class="c1"># only.</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">background_recording_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span>
<div class="viewcode-block" id="AugmentedPipeline.cacheables"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.augmented_pipeline.html#apache_beam.runners.interactive.augmented_pipeline.AugmentedPipeline.cacheables">[docs]</a> <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">-&gt;</span> <span class="n">Dict</span><span class="p">[</span><span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">,</span> <span class="n">Cacheable</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Finds all the cacheable intermediate PCollections in the pipeline with</span>
<span class="sd"> their metadata.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">c</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">watching</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">watching</span><span class="p">():</span>
<span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">val</span> <span class="ow">in</span> <span class="n">watching</span><span class="p">:</span>
<span class="k">if</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">val</span><span class="p">,</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">)</span> <span class="ow">and</span>
<span class="n">val</span><span class="o">.</span><span class="n">pipeline</span> <span class="ow">is</span> <span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span> <span class="ow">and</span>
<span class="p">(</span><span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcolls</span> <span class="ow">or</span> <span class="n">val</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcolls</span><span class="p">)):</span>
<span class="n">c</span><span class="p">[</span><span class="n">val</span><span class="p">]</span> <span class="o">=</span> <span class="n">Cacheable</span><span class="p">(</span>
<span class="n">var</span><span class="o">=</span><span class="n">key</span><span class="p">,</span>
<span class="n">pcoll</span><span class="o">=</span><span class="n">val</span><span class="p">,</span>
<span class="n">version</span><span class="o">=</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">val</span><span class="p">)),</span>
<span class="n">producer_version</span><span class="o">=</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">val</span><span class="o">.</span><span class="n">producer</span><span class="p">)))</span>
<span class="k">return</span> <span class="n">c</span></div>
<div class="viewcode-block" id="AugmentedPipeline.augment"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.augmented_pipeline.html#apache_beam.runners.interactive.augmented_pipeline.AugmentedPipeline.augment">[docs]</a> <span class="k">def</span> <span class="nf">augment</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Augments the pipeline with cache. Always calculates a new result.</span>
<span class="sd"> For a cacheable PCollection, if cache exists, read cache; else, write cache.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">pipeline</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_user_pipeline</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">()</span>
<span class="c1"># Find pcolls eligible for reading or writing cache.</span>
<span class="n">readcache_pcolls</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="k">for</span> <span class="n">pcoll</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="n">key</span> <span class="o">=</span> <span class="nb">repr</span><span class="p">(</span><span class="n">cacheable</span><span class="o">.</span><span class="n">to_key</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">_cache_manager</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="s1">&#39;full&#39;</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span> <span class="ow">and</span>
<span class="n">pcoll</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="n">readcache_pcolls</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="n">writecache_pcolls</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">_cacheables</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span><span class="o">.</span><span class="n">difference</span><span class="p">(</span><span class="n">readcache_pcolls</span><span class="p">)</span>
<span class="c1"># Wire in additional transforms to read cache and write cache.</span>
<span class="k">for</span> <span class="n">readcache_pcoll</span> <span class="ow">in</span> <span class="n">readcache_pcolls</span><span class="p">:</span>
<span class="n">ReadCache</span><span class="p">(</span>
<span class="n">pipeline</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cacheables</span><span class="p">[</span><span class="n">readcache_pcoll</span><span class="p">])</span><span class="o">.</span><span class="n">read_cache</span><span class="p">()</span>
<span class="k">for</span> <span class="n">writecache_pcoll</span> <span class="ow">in</span> <span class="n">writecache_pcolls</span><span class="p">:</span>
<span class="n">WriteCache</span><span class="p">(</span>
<span class="n">pipeline</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cacheables</span><span class="p">[</span><span class="n">writecache_pcoll</span><span class="p">])</span><span class="o">.</span><span class="n">write_cache</span><span class="p">()</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/20526): Support streaming, add</span>
<span class="c1"># pruning logic, and integrate pipeline fragment logic.</span>
<span class="k">return</span> <span class="n">pipeline</span></div></div>
</pre></div>
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/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>