blob: d977d64c7ee1d1add9705640326fd69e48218b1d [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.pipeline_analyzer &mdash; Apache Beam documentation</title>
<link rel="stylesheet" href="../../../../_static/css/theme.css" type="text/css" />
<link rel="index" title="Index"
href="../../../../genindex.html"/>
<link rel="search" title="Search" href="../../../../search.html"/>
<link rel="top" title="Apache Beam documentation" href="../../../../index.html"/>
<link rel="up" title="Module code" href="../../../index.html"/>
<script src="../../../../_static/js/modernizr.min.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search">
<a href="../../../../index.html" class="icon icon-home"> Apache Beam
</a>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.internal.html">apache_beam.internal package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.tools.html">apache_beam.tools package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.utils.html">apache_beam.utils package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.version.html">apache_beam.version module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../../index.html">Docs</a> &raquo;</li>
<li><a href="../../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.runners.interactive.pipeline_analyzer</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_analyzer</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;Analyzes and modifies the pipeline that utilize the PCollection cache.</span>
<span class="sd">This module is experimental. No backwards-compatibility guarantees.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">division</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">print_function</span>
<span class="kn">import</span> <span class="nn">collections</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="k">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="k">import</span> <span class="n">cache_manager</span> <span class="k">as</span> <span class="n">cache</span>
<div class="viewcode-block" id="PipelineAnalyzer"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineAnalyzer">[docs]</a><span class="k">class</span> <span class="nc">PipelineAnalyzer</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">cache_manager</span><span class="p">,</span> <span class="n">pipeline_proto</span><span class="p">,</span> <span class="n">underlying_runner</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="n">desired_cache_labels</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Constructor of PipelineAnanlyzer.</span>
<span class="sd"> Args:</span>
<span class="sd"> cache_manager: (CacheManager)</span>
<span class="sd"> pipeline_proto: (Pipeline proto)</span>
<span class="sd"> underlying_runner: (PipelineRunner)</span>
<span class="sd"> options: (PipelineOptions)</span>
<span class="sd"> desired_cache_labels: (Set[str]) a set of labels of the PCollection</span>
<span class="sd"> queried by the user.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span> <span class="o">=</span> <span class="n">cache_manager</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_proto</span> <span class="o">=</span> <span class="n">pipeline_proto</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_desired_cache_labels</span> <span class="o">=</span> <span class="n">desired_cache_labels</span> <span class="ow">or</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">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="bp">self</span><span class="o">.</span><span class="n">_pipeline_proto</span><span class="p">,</span>
<span class="n">runner</span><span class="o">=</span><span class="n">underlying_runner</span><span class="p">,</span>
<span class="n">options</span><span class="o">=</span><span class="n">options</span><span class="p">)</span>
<span class="c1"># context returned from to_runner_api is more informative than that returned</span>
<span class="c1"># from from_runner_api.</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">_pipeline</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">(</span>
<span class="n">return_context</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">use_fake_coders</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">_pipeline_info</span> <span class="o">=</span> <span class="n">PipelineInfo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_proto</span><span class="o">.</span><span class="n">components</span><span class="p">)</span>
<span class="c1"># Result of the analysis that can be queried by the user.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_proto_to_execute</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_top_level_referenced_pcoll_ids</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_top_level_required_transforms</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_caches_used</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">_read_cache_ids</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">_write_cache_ids</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="c1"># used for _insert_producing_transforms()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_analyzed_pcoll_ids</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">_analyze_pipeline</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_analyze_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Analyzes the pipeline and sets the variables that can be queried.</span>
<span class="sd"> This function construct Pipeline proto to execute by</span>
<span class="sd"> 1. Start from target PCollections and recursively insert the producing</span>
<span class="sd"> PTransforms of those PCollections, where the producing PTransforms are</span>
<span class="sd"> either ReadCache or PTransforms in the original pipeline.</span>
<span class="sd"> 2. Append WriteCache PTransforsm in the pipeline.</span>
<span class="sd"> After running this function, the following variables will be set:</span>
<span class="sd"> self._pipeline_proto_to_execute</span>
<span class="sd"> self._top_level_referenced_pcoll_ids</span>
<span class="sd"> self._top_level_required_transforms</span>
<span class="sd"> self._caches_used</span>
<span class="sd"> self._read_cache_ids</span>
<span class="sd"> self._write_cache_ids</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># We filter PTransforms to be executed bottom-up from these PCollections.</span>
<span class="n">desired_pcollections</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_desired_pcollections</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span><span class="p">)</span>
<span class="n">required_transforms</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">OrderedDict</span><span class="p">()</span>
<span class="n">top_level_required_transforms</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">OrderedDict</span><span class="p">()</span>
<span class="k">for</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="n">desired_pcollections</span><span class="p">:</span>
<span class="c1"># TODO(qinyeli): Collections consumed by no-output transforms.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_insert_producing_transforms</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">,</span>
<span class="n">required_transforms</span><span class="p">,</span>
<span class="n">top_level_required_transforms</span><span class="p">)</span>
<span class="n">top_level_referenced_pcoll_ids</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_referenced_pcoll_ids</span><span class="p">(</span>
<span class="n">top_level_required_transforms</span><span class="p">)</span>
<span class="k">for</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span><span class="o">.</span><span class="n">all_pcollections</span><span class="p">():</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="n">top_level_referenced_pcoll_ids</span><span class="p">:</span>
<span class="k">continue</span>
<span class="k">if</span> <span class="p">(</span><span class="n">pcoll_id</span> <span class="ow">in</span> <span class="n">desired_pcollections</span>
<span class="ow">and</span> <span class="ow">not</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_caches_used</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_insert_caching_transforms</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">,</span>
<span class="n">required_transforms</span><span class="p">,</span>
<span class="n">top_level_required_transforms</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">_cache_manager</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span>
<span class="s1">&#39;sample&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span><span class="o">.</span><span class="n">cache_label</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_insert_caching_transforms</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">,</span>
<span class="n">required_transforms</span><span class="p">,</span>
<span class="n">top_level_required_transforms</span><span class="p">,</span>
<span class="n">sample</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">required_transforms</span><span class="p">[</span><span class="s1">&#39;_root&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">PTransform</span><span class="p">(</span>
<span class="n">subtransforms</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">top_level_required_transforms</span><span class="p">))</span>
<span class="n">referenced_pcoll_ids</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_referenced_pcoll_ids</span><span class="p">(</span>
<span class="n">required_transforms</span><span class="p">)</span>
<span class="n">referenced_pcollections</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="n">referenced_pcoll_ids</span><span class="p">:</span>
<span class="n">obj</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">pcollections</span><span class="o">.</span><span class="n">get_by_id</span><span class="p">(</span><span class="n">pcoll_id</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">_context</span><span class="o">.</span><span class="n">pcollections</span><span class="o">.</span><span class="n">get_proto</span><span class="p">(</span><span class="n">obj</span><span class="p">)</span>
<span class="n">referenced_pcollections</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span> <span class="o">=</span> <span class="n">proto</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="p">[</span><span class="s1">&#39;_root&#39;</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">transforms</span><span class="p">,</span>
<span class="n">required_transforms</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">referenced_pcollections</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="bp">self</span><span class="o">.</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="bp">self</span><span class="o">.</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="bp">self</span><span class="o">.</span><span class="n">_pipeline_proto_to_execute</span> <span class="o">=</span> <span class="n">pipeline_to_execute</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_top_level_referenced_pcoll_ids</span> <span class="o">=</span> <span class="n">top_level_referenced_pcoll_ids</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_top_level_required_transforms</span> <span class="o">=</span> <span class="n">top_level_required_transforms</span>
<span class="c1"># -------------------------------------------------------------------------- #</span>
<span class="c1"># Getters</span>
<span class="c1"># -------------------------------------------------------------------------- #</span>
<div class="viewcode-block" id="PipelineAnalyzer.pipeline_info"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineAnalyzer.pipeline_info">[docs]</a> <span class="k">def</span> <span class="nf">pipeline_info</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Return PipelineInfo of the original pipeline.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span></div>
<div class="viewcode-block" id="PipelineAnalyzer.pipeline_proto_to_execute"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineAnalyzer.pipeline_proto_to_execute">[docs]</a> <span class="k">def</span> <span class="nf">pipeline_proto_to_execute</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns Pipeline proto to be executed.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_proto_to_execute</span></div>
<div class="viewcode-block" id="PipelineAnalyzer.tl_referenced_pcoll_ids"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineAnalyzer.tl_referenced_pcoll_ids">[docs]</a> <span class="k">def</span> <span class="nf">tl_referenced_pcoll_ids</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a set of PCollection IDs referenced by top level PTransforms.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_top_level_referenced_pcoll_ids</span></div>
<div class="viewcode-block" id="PipelineAnalyzer.tl_required_trans_ids"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineAnalyzer.tl_required_trans_ids">[docs]</a> <span class="k">def</span> <span class="nf">tl_required_trans_ids</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a set of required top level PTransform IDs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_top_level_required_transforms</span><span class="p">)</span></div>
<div class="viewcode-block" id="PipelineAnalyzer.caches_used"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineAnalyzer.caches_used">[docs]</a> <span class="k">def</span> <span class="nf">caches_used</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a set of PCollection IDs to read from cache.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_caches_used</span></div>
<div class="viewcode-block" id="PipelineAnalyzer.read_cache_ids"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineAnalyzer.read_cache_ids">[docs]</a> <span class="k">def</span> <span class="nf">read_cache_ids</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Return a set of ReadCache PTransform IDs inserted.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_cache_ids</span></div>
<div class="viewcode-block" id="PipelineAnalyzer.write_cache_ids"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineAnalyzer.write_cache_ids">[docs]</a> <span class="k">def</span> <span class="nf">write_cache_ids</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Return a set of WriteCache PTransform IDs inserted.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_write_cache_ids</span></div>
<span class="c1"># -------------------------------------------------------------------------- #</span>
<span class="c1"># Helper methods for _analyze_pipeline()</span>
<span class="c1"># -------------------------------------------------------------------------- #</span>
<span class="k">def</span> <span class="nf">_insert_producing_transforms</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span>
<span class="n">pcoll_id</span><span class="p">,</span>
<span class="n">required_transforms</span><span class="p">,</span>
<span class="n">top_level_required_transforms</span><span class="p">,</span>
<span class="n">leaf</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Inserts PTransforms producing the given PCollection into the dicts.</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll_id: (str)</span>
<span class="sd"> required_transforms: (Dict[str, PTransform proto])</span>
<span class="sd"> top_level_required_transforms: (Dict[str, PTransform proto])</span>
<span class="sd"> leaf: (bool) whether the PCollection should be read from cache if the</span>
<span class="sd"> cache exists.</span>
<span class="sd"> Modifies:</span>
<span class="sd"> required_transforms</span>
<span class="sd"> top_level_required_transforms</span>
<span class="sd"> self._read_cache_ids</span>
<span class="sd"> &quot;&quot;&quot;</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">_analyzed_pcoll_ids</span><span class="p">:</span>
<span class="k">return</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_analyzed_pcoll_ids</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="n">cache_label</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span><span class="o">.</span><span class="n">cache_label</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="s1">&#39;full&#39;</span><span class="p">,</span> <span class="n">cache_label</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">leaf</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_caches_used</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="n">cache_label</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span><span class="o">.</span><span class="n">cache_label</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="n">dummy_pcoll</span> <span class="o">=</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="s1">&#39;Load</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">cache_label</span> <span class="o">&gt;&gt;</span> <span class="n">cache</span><span class="o">.</span><span class="n">ReadCache</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="p">,</span> <span class="n">cache_label</span><span class="p">))</span>
<span class="n">read_cache</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_top_level_producer</span><span class="p">(</span><span class="n">dummy_pcoll</span><span class="p">)</span>
<span class="n">read_cache_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span><span class="n">read_cache</span><span class="p">)</span>
<span class="n">read_cache_proto</span> <span class="o">=</span> <span class="n">read_cache</span><span class="o">.</span><span class="n">to_runner_api</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="n">read_cache_proto</span><span class="o">.</span><span class="n">outputs</span><span class="p">[</span><span class="s1">&#39;None&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">pcoll_id</span>
<span class="n">top_level_required_transforms</span><span class="p">[</span><span class="n">read_cache_id</span><span class="p">]</span> <span class="o">=</span> <span class="n">read_cache_proto</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_read_cache_ids</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">read_cache_id</span><span class="p">)</span>
<span class="k">for</span> <span class="n">transform</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_include_subtransforms</span><span class="p">(</span><span class="n">read_cache</span><span class="p">):</span>
<span class="n">transform_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span><span class="n">transform</span><span class="p">)</span>
<span class="n">transform_proto</span> <span class="o">=</span> <span class="n">transform</span><span class="o">.</span><span class="n">to_runner_api</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="k">if</span> <span class="n">dummy_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">values</span><span class="p">():</span>
<span class="n">transform_proto</span><span class="o">.</span><span class="n">outputs</span><span class="p">[</span><span class="s1">&#39;None&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">pcoll_id</span>
<span class="n">required_transforms</span><span class="p">[</span><span class="n">transform_id</span><span class="p">]</span> <span class="o">=</span> <span class="n">transform_proto</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">pcoll</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">pcollections</span><span class="o">.</span><span class="n">get_by_id</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="n">top_level_transform</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_top_level_producer</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span>
<span class="k">for</span> <span class="n">transform</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_include_subtransforms</span><span class="p">(</span><span class="n">top_level_transform</span><span class="p">):</span>
<span class="n">transform_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span><span class="n">transform</span><span class="p">)</span>
<span class="n">transform_proto</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_proto</span><span class="p">(</span><span class="n">transform</span><span class="p">)</span>
<span class="c1"># Inserting ancestor PTransforms.</span>
<span class="k">for</span> <span class="n">input_id</span> <span class="ow">in</span> <span class="n">transform_proto</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="bp">self</span><span class="o">.</span><span class="n">_insert_producing_transforms</span><span class="p">(</span><span class="n">input_id</span><span class="p">,</span>
<span class="n">required_transforms</span><span class="p">,</span>
<span class="n">top_level_required_transforms</span><span class="p">)</span>
<span class="n">required_transforms</span><span class="p">[</span><span class="n">transform_id</span><span class="p">]</span> <span class="o">=</span> <span class="n">transform_proto</span>
<span class="c1"># Must be inserted after inserting ancestor PTransforms.</span>
<span class="n">top_level_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span><span class="n">top_level_transform</span><span class="p">)</span>
<span class="n">top_level_proto</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_proto</span><span class="p">(</span><span class="n">top_level_transform</span><span class="p">)</span>
<span class="n">top_level_required_transforms</span><span class="p">[</span><span class="n">top_level_id</span><span class="p">]</span> <span class="o">=</span> <span class="n">top_level_proto</span>
<span class="k">def</span> <span class="nf">_insert_caching_transforms</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span>
<span class="n">pcoll_id</span><span class="p">,</span>
<span class="n">required_transforms</span><span class="p">,</span>
<span class="n">top_level_required_transforms</span><span class="p">,</span>
<span class="n">sample</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Inserts PTransforms caching the given PCollection into the dicts.</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll_id: (str)</span>
<span class="sd"> required_transforms: (Dict[str, PTransform proto])</span>
<span class="sd"> top_level_required_transforms: (Dict[str, PTransform proto])</span>
<span class="sd"> sample: (bool) whether to cache sample or cache full.</span>
<span class="sd"> Modifies:</span>
<span class="sd"> required_transforms</span>
<span class="sd"> top_level_required_transforms</span>
<span class="sd"> self._write_cache_ids</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">cache_label</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_info</span><span class="o">.</span><span class="n">cache_label</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="n">pcoll</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">pcollections</span><span class="o">.</span><span class="n">get_by_id</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">sample</span><span class="p">:</span>
<span class="n">pdone</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">&#39;CacheFull</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">cache_label</span> <span class="o">&gt;&gt;</span> <span class="n">cache</span><span class="o">.</span><span class="n">WriteCache</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="p">,</span> <span class="n">cache_label</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">pdone</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">&#39;CacheSample</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">cache_label</span> <span class="o">&gt;&gt;</span> <span class="n">cache</span><span class="o">.</span><span class="n">WriteCache</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_manager</span><span class="p">,</span> <span class="n">cache_label</span><span class="p">,</span> <span class="n">sample</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="n">sample_size</span><span class="o">=</span><span class="mi">10</span><span class="p">)</span>
<span class="n">write_cache</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_top_level_producer</span><span class="p">(</span><span class="n">pdone</span><span class="p">)</span>
<span class="n">write_cache_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span><span class="n">write_cache</span><span class="p">)</span>
<span class="n">write_cache_proto</span> <span class="o">=</span> <span class="n">write_cache</span><span class="o">.</span><span class="n">to_runner_api</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="n">top_level_required_transforms</span><span class="p">[</span><span class="n">write_cache_id</span><span class="p">]</span> <span class="o">=</span> <span class="n">write_cache_proto</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_write_cache_ids</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">write_cache_id</span><span class="p">)</span>
<span class="k">for</span> <span class="n">transform</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_include_subtransforms</span><span class="p">(</span><span class="n">write_cache</span><span class="p">):</span>
<span class="n">transform_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span><span class="n">transform</span><span class="p">)</span>
<span class="n">transform_proto</span> <span class="o">=</span> <span class="n">transform</span><span class="o">.</span><span class="n">to_runner_api</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="n">required_transforms</span><span class="p">[</span><span class="n">transform_id</span><span class="p">]</span> <span class="o">=</span> <span class="n">transform_proto</span>
<span class="k">def</span> <span class="nf">_desired_pcollections</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline_info</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns IDs of desired (queried or leaf) PCollections.</span>
<span class="sd"> Args:</span>
<span class="sd"> pipeline_info: (PipelineInfo) info of the original pipeline.</span>
<span class="sd"> Returns:</span>
<span class="sd"> (Set[str]) a set of PCollections IDs of either leaf PCollections or</span>
<span class="sd"> PCollections referenced by the user. These PCollections should be cached</span>
<span class="sd"> at the end of pipeline execution.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">desired_pcollections</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">pipeline_info</span><span class="o">.</span><span class="n">leaf_pcollections</span><span class="p">())</span>
<span class="k">for</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="n">pipeline_info</span><span class="o">.</span><span class="n">all_pcollections</span><span class="p">():</span>
<span class="n">cache_label</span> <span class="o">=</span> <span class="n">pipeline_info</span><span class="o">.</span><span class="n">cache_label</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="k">if</span> <span class="n">cache_label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_desired_cache_labels</span><span class="p">:</span>
<span class="n">desired_pcollections</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="k">return</span> <span class="n">desired_pcollections</span>
<span class="k">def</span> <span class="nf">_referenced_pcoll_ids</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">required_transforms</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns PCollection IDs referenced in the given transforms.</span>
<span class="sd"> Args:</span>
<span class="sd"> transforms: (Dict[str, PTransform proto]) mapping ID to protos.</span>
<span class="sd"> Returns:</span>
<span class="sd"> (Set[str]) PCollection IDs referenced as either input or output in the</span>
<span class="sd"> given transforms.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">referenced_pcoll_ids</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="k">for</span> <span class="n">transform_proto</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">pcoll_id</span> <span class="ow">in</span> <span class="n">transform_proto</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="n">referenced_pcoll_ids</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="k">for</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="n">transform_proto</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">referenced_pcoll_ids</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span>
<span class="k">return</span> <span class="n">referenced_pcoll_ids</span>
<span class="k">def</span> <span class="nf">_top_level_producer</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="sd">&quot;&quot;&quot;Given a PCollection, returns the top level producing PTransform.</span>
<span class="sd"> Args:</span>
<span class="sd"> pcoll: (PCollection)</span>
<span class="sd"> Returns:</span>
<span class="sd"> (PTransform) top level producing PTransform of pcoll.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">top_level_transform</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">top_level_transform</span><span class="o">.</span><span class="n">parent</span><span class="o">.</span><span class="n">parent</span><span class="p">:</span>
<span class="n">top_level_transform</span> <span class="o">=</span> <span class="n">top_level_transform</span><span class="o">.</span><span class="n">parent</span>
<span class="k">return</span> <span class="n">top_level_transform</span>
<span class="k">def</span> <span class="nf">_include_subtransforms</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="sd">&quot;&quot;&quot;Depth-first yield the PTransform itself and its sub transforms.</span>
<span class="sd"> Args:</span>
<span class="sd"> transform: (PTransform)</span>
<span class="sd"> Yields:</span>
<span class="sd"> The input PTransform itself and all its sub transforms.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">yield</span> <span class="n">transform</span>
<span class="k">for</span> <span class="n">subtransform</span> <span class="ow">in</span> <span class="n">transform</span><span class="o">.</span><span class="n">parts</span><span class="p">[::</span><span class="o">-</span><span class="mi">1</span><span class="p">]:</span>
<span class="k">for</span> <span class="n">yielded</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_include_subtransforms</span><span class="p">(</span><span class="n">subtransform</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">yielded</span></div>
<div class="viewcode-block" id="PipelineInfo"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineInfo">[docs]</a><span class="k">class</span> <span class="nc">PipelineInfo</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Provides access to pipeline metadata.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">proto</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_proto</span> <span class="o">=</span> <span class="n">proto</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_producers</span> <span class="o">=</span> <span class="p">{}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_consumers</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">)</span>
<span class="k">for</span> <span class="n">transform_id</span><span class="p">,</span> <span class="n">transform_proto</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_proto</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">transform_proto</span><span class="o">.</span><span class="n">subtransforms</span><span class="p">:</span>
<span class="k">continue</span>
<span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="n">transform_proto</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="bp">self</span><span class="o">.</span><span class="n">_producers</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span> <span class="o">=</span> <span class="n">transform_id</span><span class="p">,</span> <span class="n">tag</span>
<span class="k">for</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="n">transform_proto</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="bp">self</span><span class="o">.</span><span class="n">_consumers</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">transform_id</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_derivations</span> <span class="o">=</span> <span class="p">{}</span>
<div class="viewcode-block" id="PipelineInfo.all_pcollections"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineInfo.all_pcollections">[docs]</a> <span class="k">def</span> <span class="nf">all_pcollections</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_proto</span><span class="o">.</span><span class="n">pcollections</span><span class="o">.</span><span class="n">keys</span><span class="p">()</span></div>
<div class="viewcode-block" id="PipelineInfo.leaf_pcollections"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineInfo.leaf_pcollections">[docs]</a> <span class="k">def</span> <span class="nf">leaf_pcollections</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">for</span> <span class="n">pcoll_id</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_proto</span><span class="o">.</span><span class="n">pcollections</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">_consumers</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]:</span>
<span class="k">yield</span> <span class="n">pcoll_id</span></div>
<div class="viewcode-block" id="PipelineInfo.producer"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineInfo.producer">[docs]</a> <span class="k">def</span> <span class="nf">producer</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll_id</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_producers</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span></div>
<div class="viewcode-block" id="PipelineInfo.cache_label"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineInfo.cache_label">[docs]</a> <span class="k">def</span> <span class="nf">cache_label</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll_id</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns the cache label given the PCollection ID.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_derivation</span><span class="p">(</span><span class="n">pcoll_id</span><span class="p">)</span><span class="o">.</span><span class="n">cache_label</span><span class="p">()</span></div>
<span class="k">def</span> <span class="nf">_derivation</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll_id</span><span class="p">):</span>
<span class="k">if</span> <span class="n">pcoll_id</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_derivations</span><span class="p">:</span>
<span class="n">transform_id</span><span class="p">,</span> <span class="n">output_tag</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_producers</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span>
<span class="n">transform_proto</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_proto</span><span class="o">.</span><span class="n">transforms</span><span class="p">[</span><span class="n">transform_id</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_derivations</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">Derivation</span><span class="p">({</span>
<span class="n">input_tag</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_derivation</span><span class="p">(</span><span class="n">input_id</span><span class="p">)</span>
<span class="k">for</span> <span class="n">input_tag</span><span class="p">,</span> <span class="n">input_id</span> <span class="ow">in</span> <span class="n">transform_proto</span><span class="o">.</span><span class="n">inputs</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">},</span> <span class="n">transform_proto</span><span class="p">,</span> <span class="n">output_tag</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_derivations</span><span class="p">[</span><span class="n">pcoll_id</span><span class="p">]</span>
<div class="viewcode-block" id="PipelineInfo.Derivation"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineInfo.Derivation">[docs]</a> <span class="k">class</span> <span class="nc">Derivation</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Records derivation info of a PCollection. Helper for PipelineInfo.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">inputs</span><span class="p">,</span> <span class="n">transform_proto</span><span class="p">,</span> <span class="n">output_tag</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Constructor of Derivation.</span>
<span class="sd"> Args:</span>
<span class="sd"> inputs: (Dict[str, Derivation]) maps PCollection names to Derivations.</span>
<span class="sd"> transform_proto: (Transform proto) the producing PTransform.</span>
<span class="sd"> output_tag: (str) local name of the PCollection in analysis.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_inputs</span> <span class="o">=</span> <span class="n">inputs</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_info</span> <span class="o">=</span> <span class="p">{</span>
<span class="c1"># TODO(qinyeli): remove name field when collision is resolved.</span>
<span class="s1">&#39;name&#39;</span><span class="p">:</span> <span class="n">transform_proto</span><span class="o">.</span><span class="n">unique_name</span><span class="p">,</span>
<span class="s1">&#39;urn&#39;</span><span class="p">:</span> <span class="n">transform_proto</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="s1">&#39;payload&#39;</span><span class="p">:</span> <span class="n">transform_proto</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">payload</span><span class="o">.</span><span class="n">decode</span><span class="p">(</span><span class="s1">&#39;latin1&#39;</span><span class="p">)</span>
<span class="p">}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_output_tag</span> <span class="o">=</span> <span class="n">output_tag</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_hash</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">Derivation</span><span class="p">):</span>
<span class="c1"># pylint: disable=protected-access</span>
<span class="k">return</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_inputs</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">_inputs</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_transform_info</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">_transform_info</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">__ne__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="c1"># TODO(BEAM-5949): Needed for Python 2 compatibility.</span>
<span class="k">return</span> <span class="ow">not</span> <span class="bp">self</span> <span class="o">==</span> <span class="n">other</span>
<span class="k">def</span> <span class="nf">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_hash</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_hash</span> <span class="o">=</span> <span class="p">(</span><span class="nb">hash</span><span class="p">(</span><span class="nb">tuple</span><span class="p">(</span><span class="nb">sorted</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_transform_info</span><span class="o">.</span><span class="n">items</span><span class="p">())))</span>
<span class="o">+</span> <span class="nb">sum</span><span class="p">(</span><span class="nb">hash</span><span class="p">(</span><span class="n">tag</span><span class="p">)</span> <span class="o">*</span> <span class="nb">hash</span><span class="p">(</span><span class="nb">input</span><span class="p">)</span>
<span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="nb">input</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_inputs</span><span class="o">.</span><span class="n">items</span><span class="p">())</span>
<span class="o">+</span> <span class="nb">hash</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_output_tag</span><span class="p">))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_hash</span>
<div class="viewcode-block" id="PipelineInfo.Derivation.cache_label"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineInfo.Derivation.cache_label">[docs]</a> <span class="k">def</span> <span class="nf">cache_label</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># TODO(qinyeli): Collision resistance?</span>
<span class="k">return</span> <span class="s1">&#39;Pcoll-</span><span class="si">%x</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="nb">abs</span><span class="p">(</span><span class="nb">hash</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span></div>
<div class="viewcode-block" id="PipelineInfo.Derivation.json"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.PipelineInfo.Derivation.json">[docs]</a> <span class="k">def</span> <span class="nf">json</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="p">{</span>
<span class="s1">&#39;inputs&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_inputs</span><span class="p">,</span>
<span class="s1">&#39;transform&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_transform_info</span><span class="p">,</span>
<span class="s1">&#39;output_tag&#39;</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_output_tag</span>
<span class="p">}</span></div>
<span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">json</span><span class="p">())</span></div></div>
<span class="c1"># TODO(qinyeli) move to proto_utils</span>
<div class="viewcode-block" id="set_proto_map"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.pipeline_analyzer.html#apache_beam.runners.interactive.pipeline_analyzer.set_proto_map">[docs]</a><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></div>
</pre></div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright .
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
var DOCUMENTATION_OPTIONS = {
URL_ROOT:'../../../../',
VERSION:'',
COLLAPSE_INDEX:false,
FILE_SUFFIX:'.html',
HAS_SOURCE: true,
SOURCELINK_SUFFIX: '.txt'
};
</script>
<script type="text/javascript" src="../../../../_static/jquery.js"></script>
<script type="text/javascript" src="../../../../_static/underscore.js"></script>
<script type="text/javascript" src="../../../../_static/doctools.js"></script>
<script type="text/javascript" src="../../../../_static/js/theme.js"></script>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.StickyNav.enable();
});
</script>
</body>
</html>