| |
| |
| <!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.recording_manager — 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.recording_manager</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.recording_manager</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="kn">import</span> <span class="nn">logging</span> |
| <span class="kn">import</span> <span class="nn">threading</span> |
| <span class="kn">import</span> <span class="nn">time</span> |
| <span class="kn">import</span> <span class="nn">warnings</span> |
| |
| <span class="kn">import</span> <span class="nn">pandas</span> <span class="k">as</span> <span class="nn">pd</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.dataframe.frame_base</span> <span class="kn">import</span> <span class="n">DeferredBase</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">background_caching_job</span> <span class="k">as</span> <span class="n">bcj</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">interactive_runner</span> <span class="k">as</span> <span class="n">ir</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">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">CacheKey</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.runner</span> <span class="kn">import</span> <span class="n">PipelineState</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="ElementStream"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.ElementStream">[docs]</a><span class="k">class</span> <span class="nc">ElementStream</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""A stream of elements from a given PCollection."""</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">pcoll</span><span class="p">,</span> <span class="c1"># type: beam.pvalue.PCollection</span> |
| <span class="n">var</span><span class="p">,</span> <span class="c1"># type: str</span> |
| <span class="n">cache_key</span><span class="p">,</span> <span class="c1"># type: str</span> |
| <span class="n">max_n</span><span class="p">,</span> <span class="c1"># type: int</span> |
| <span class="n">max_duration_secs</span> <span class="c1"># type: float</span> |
| <span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pcoll</span> <span class="o">=</span> <span class="n">pcoll</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache_key</span> <span class="o">=</span> <span class="n">cache_key</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_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">pcoll</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">_var</span> <span class="o">=</span> <span class="n">var</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_n</span> <span class="o">=</span> <span class="n">max_n</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_duration_secs</span> <span class="o">=</span> <span class="n">max_duration_secs</span> |
| |
| <span class="c1"># A small state variable that when True, indicates that no more new elements</span> |
| <span class="c1"># will be yielded if read() is called again.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_done</span> <span class="o">=</span> <span class="kc">False</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">var</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> str</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns the variable named that defined this PCollection."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_var</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">pcoll</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> beam.pvalue.PCollection</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns the PCollection that supplies this stream with data."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pcoll</span> |
| |
| <span class="nd">@property</span> |
| <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="c1"># type: () -> str</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns the cache key for this stream."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_key</span> |
| |
| <div class="viewcode-block" id="ElementStream.display_id"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.ElementStream.display_id">[docs]</a> <span class="k">def</span> <span class="nf">display_id</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">suffix</span><span class="p">):</span> |
| <span class="c1"># type: (str) -> str</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns a unique id able to be displayed in a web browser."""</span> |
| <span class="k">return</span> <span class="n">utils</span><span class="o">.</span><span class="n">obfuscate</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_key</span><span class="p">,</span> <span class="n">suffix</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="ElementStream.is_computed"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.ElementStream.is_computed">[docs]</a> <span class="k">def</span> <span class="nf">is_computed</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> boolean # noqa: F821</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns True if no more elements will be recorded."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</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></div> |
| |
| <div class="viewcode-block" id="ElementStream.is_done"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.ElementStream.is_done">[docs]</a> <span class="k">def</span> <span class="nf">is_done</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> boolean # noqa: F821</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns True if no more new elements will be yielded."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_done</span></div> |
| |
| <div class="viewcode-block" id="ElementStream.read"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.ElementStream.read">[docs]</a> <span class="k">def</span> <span class="nf">read</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">tail</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="c1"># type: (boolean) -> Any # noqa: F821</span> |
| |
| <span class="w"> </span><span class="sd">"""Reads the elements currently recorded."""</span> |
| |
| <span class="c1"># Get the cache manager and wait until the file exists.</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">_pipeline</span><span class="p">)</span> |
| |
| <span class="c1"># Retrieve the coder for the particular PCollection which will be used to</span> |
| <span class="c1"># decode elements read from cache.</span> |
| <span class="n">coder</span> <span class="o">=</span> <span class="n">cache_manager</span><span class="o">.</span><span class="n">load_pcoder</span><span class="p">(</span><span class="s1">'full'</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_key</span><span class="p">)</span> |
| |
| <span class="c1"># Read the elements from the cache.</span> |
| <span class="c1"># Import limiters here to prevent a circular import.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.options.capture_limiters</span> <span class="kn">import</span> <span class="n">CountLimiter</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.options.capture_limiters</span> <span class="kn">import</span> <span class="n">ProcessingTimeLimiter</span> |
| <span class="n">reader</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">cache_manager</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="s1">'full'</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_key</span><span class="p">,</span> <span class="n">tail</span><span class="o">=</span><span class="n">tail</span><span class="p">)</span> |
| |
| <span class="c1"># Because a single TestStreamFileRecord can yield multiple elements, we</span> |
| <span class="c1"># limit the count again here in the to_element_list call.</span> |
| <span class="c1">#</span> |
| <span class="c1"># There are two ways of exiting this loop either a limiter was triggered or</span> |
| <span class="c1"># all elements from the cache were read. In the latter situation, it may be</span> |
| <span class="c1"># the case that the pipeline was still running. Thus, another invocation of</span> |
| <span class="c1"># `read` will yield new elements.</span> |
| <span class="n">count_limiter</span> <span class="o">=</span> <span class="n">CountLimiter</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">)</span> |
| <span class="n">time_limiter</span> <span class="o">=</span> <span class="n">ProcessingTimeLimiter</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_duration_secs</span><span class="p">)</span> |
| <span class="n">limiters</span> <span class="o">=</span> <span class="p">(</span><span class="n">count_limiter</span><span class="p">,</span> <span class="n">time_limiter</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">e</span> <span class="ow">in</span> <span class="n">utils</span><span class="o">.</span><span class="n">to_element_list</span><span class="p">(</span><span class="n">reader</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">,</span> |
| <span class="n">include_window_info</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">n</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_n</span><span class="p">,</span> |
| <span class="n">include_time_events</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| |
| <span class="c1"># From the to_element_list we either get TestStreamPayload.Events if</span> |
| <span class="c1"># include_time_events or decoded elements from the reader. Make sure we</span> |
| <span class="c1"># only count the decoded elements to break early.</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">e</span><span class="p">,</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">TestStreamPayload</span><span class="o">.</span><span class="n">Event</span><span class="p">):</span> |
| <span class="n">time_limiter</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">e</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">count_limiter</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">e</span><span class="p">)</span> |
| <span class="k">yield</span> <span class="n">e</span> |
| |
| <span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="n">l</span><span class="o">.</span><span class="n">is_triggered</span><span class="p">()</span> <span class="k">for</span> <span class="n">l</span> <span class="ow">in</span> <span class="n">limiters</span><span class="p">):</span> |
| <span class="k">break</span> |
| |
| <span class="c1"># A limiter being triggered means that we have fulfilled the user's request.</span> |
| <span class="c1"># This implies that reading from the cache again won't yield any new</span> |
| <span class="c1"># elements. WLOG, this applies to the user pipeline being terminated.</span> |
| <span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="n">l</span><span class="o">.</span><span class="n">is_triggered</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">l</span> <span class="ow">in</span> <span class="n">limiters</span><span class="p">)</span> <span class="ow">or</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">is_terminated</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">_done</span> <span class="o">=</span> <span class="kc">True</span></div></div> |
| |
| |
| <div class="viewcode-block" id="Recording"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.Recording">[docs]</a><span class="k">class</span> <span class="nc">Recording</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""A group of PCollections from a given pipeline run."""</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="c1"># type: beam.Pipeline</span> |
| <span class="n">pcolls</span><span class="p">,</span> <span class="c1"># type: List[beam.pvalue.PCollection] # noqa: F821</span> |
| <span class="n">result</span><span class="p">,</span> <span class="c1"># type: beam.runner.PipelineResult</span> |
| <span class="n">max_n</span><span class="p">,</span> <span class="c1"># type: int</span> |
| <span class="n">max_duration_secs</span><span class="p">,</span> <span class="c1"># type: float</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">_result</span> <span class="o">=</span> <span class="n">result</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_result_lock</span> <span class="o">=</span> <span class="n">threading</span><span class="o">.</span><span class="n">Lock</span><span class="p">()</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="n">pcoll_var</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">pcoll</span><span class="p">:</span> <span class="p">{</span><span class="n">v</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">utils</span><span class="o">.</span><span class="n">pcoll_by_name</span><span class="p">()</span><span class="o">.</span><span class="n">items</span><span class="p">()}</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="kc">None</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_streams</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">pcoll</span><span class="p">:</span> <span class="n">ElementStream</span><span class="p">(</span> |
| <span class="n">pcoll</span><span class="p">,</span> |
| <span class="n">pcoll_var</span><span class="p">(</span><span class="n">pcoll</span><span class="p">),</span> |
| <span class="n">CacheKey</span><span class="o">.</span><span class="n">from_pcoll</span><span class="p">(</span><span class="n">pcoll_var</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="o">.</span><span class="n">to_str</span><span class="p">(),</span> |
| <span class="n">max_n</span><span class="p">,</span> |
| <span class="n">max_duration_secs</span><span class="p">)</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="bp">self</span><span class="o">.</span><span class="n">_start</span> <span class="o">=</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_duration_secs</span> <span class="o">=</span> <span class="n">max_duration_secs</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_set_computed</span> <span class="o">=</span> <span class="n">bcj</span><span class="o">.</span><span class="n">is_cache_complete</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">user_pipeline</span><span class="p">)))</span> |
| |
| <span class="c1"># Run a separate thread for marking the PCollections done. This is because</span> |
| <span class="c1"># the pipeline run may be asynchronous.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_mark_computed</span> <span class="o">=</span> <span class="n">threading</span><span class="o">.</span><span class="n">Thread</span><span class="p">(</span><span class="n">target</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_mark_all_computed</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_mark_computed</span><span class="o">.</span><span class="n">daemon</span> <span class="o">=</span> <span class="kc">True</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_mark_computed</span><span class="o">.</span><span class="n">start</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="nf">_mark_all_computed</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> None</span> |
| |
| <span class="w"> </span><span class="sd">"""Marks all the PCollections upon a successful pipeline run."""</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="p">:</span> |
| <span class="k">return</span> |
| |
| <span class="k">while</span> <span class="ow">not</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">is_terminal</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="o">.</span><span class="n">state</span><span class="p">):</span> |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_result_lock</span><span class="p">:</span> |
| <span class="n">bcj</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_background_caching_job</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="k">if</span> <span class="n">bcj</span> <span class="ow">and</span> <span class="n">bcj</span><span class="o">.</span><span class="n">is_done</span><span class="p">():</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="o">.</span><span class="n">wait_until_finish</span><span class="p">()</span> |
| |
| <span class="k">elif</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> <span class="o">-</span> <span class="bp">self</span><span class="o">.</span><span class="n">_start</span> <span class="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_duration_secs</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="o">.</span><span class="n">cancel</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="o">.</span><span class="n">wait_until_finish</span><span class="p">()</span> |
| |
| <span class="k">elif</span> <span class="nb">all</span><span class="p">(</span><span class="n">s</span><span class="o">.</span><span class="n">is_done</span><span class="p">()</span> <span class="k">for</span> <span class="n">s</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_streams</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">_result</span><span class="o">.</span><span class="n">cancel</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="o">.</span><span class="n">wait_until_finish</span><span class="p">()</span> |
| |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="mf">0.1</span><span class="p">)</span> |
| |
| <span class="c1"># Mark the PCollection as computed so that Interactive Beam wouldn't need to</span> |
| <span class="c1"># re-compute.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="o">.</span><span class="n">state</span> <span class="ow">is</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">DONE</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_set_computed</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">mark_pcollection_computed</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pcolls</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="Recording.is_computed"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.Recording.is_computed">[docs]</a> <span class="k">def</span> <span class="nf">is_computed</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> boolean # noqa: F821</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns True if all PCollections are computed."""</span> |
| <span class="k">return</span> <span class="nb">all</span><span class="p">(</span><span class="n">s</span><span class="o">.</span><span class="n">is_computed</span><span class="p">()</span> <span class="k">for</span> <span class="n">s</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_streams</span><span class="o">.</span><span class="n">values</span><span class="p">())</span></div> |
| |
| <div class="viewcode-block" id="Recording.stream"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.Recording.stream">[docs]</a> <span class="k">def</span> <span class="nf">stream</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="c1"># type: (beam.pvalue.PCollection) -> ElementStream</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns an ElementStream for a given PCollection."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_streams</span><span class="p">[</span><span class="n">pcoll</span><span class="p">]</span></div> |
| |
| <div class="viewcode-block" id="Recording.computed"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.Recording.computed">[docs]</a> <span class="k">def</span> <span class="nf">computed</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> None</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns all computed ElementStreams."""</span> |
| <span class="k">return</span> <span class="p">{</span><span class="n">p</span><span class="p">:</span> <span class="n">s</span> <span class="k">for</span> <span class="n">p</span><span class="p">,</span> <span class="n">s</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_streams</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> <span class="k">if</span> <span class="n">s</span><span class="o">.</span><span class="n">is_computed</span><span class="p">()}</span></div> |
| |
| <div class="viewcode-block" id="Recording.uncomputed"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.Recording.uncomputed">[docs]</a> <span class="k">def</span> <span class="nf">uncomputed</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> None</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns all uncomputed ElementStreams."""</span> |
| <span class="k">return</span> <span class="p">{</span><span class="n">p</span><span class="p">:</span> <span class="n">s</span> <span class="k">for</span> <span class="n">p</span><span class="p">,</span> <span class="n">s</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_streams</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> <span class="k">if</span> <span class="ow">not</span> <span class="n">s</span><span class="o">.</span><span class="n">is_computed</span><span class="p">()}</span></div> |
| |
| <div class="viewcode-block" id="Recording.cancel"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.Recording.cancel">[docs]</a> <span class="k">def</span> <span class="nf">cancel</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> None</span> |
| |
| <span class="w"> </span><span class="sd">"""Cancels the recording."""</span> |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_result_lock</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="o">.</span><span class="n">cancel</span><span class="p">()</span></div> |
| |
| <div class="viewcode-block" id="Recording.wait_until_finish"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.Recording.wait_until_finish">[docs]</a> <span class="k">def</span> <span class="nf">wait_until_finish</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> None</span> |
| |
| <span class="w"> </span><span class="sd">"""Waits until the pipeline is done and returns the final state.</span> |
| |
| <span class="sd"> This also marks any PCollections as computed right away if the pipeline is</span> |
| <span class="sd"> successful.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">runners</span><span class="o">.</span><span class="n">runner</span><span class="o">.</span><span class="n">PipelineState</span><span class="o">.</span><span class="n">DONE</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_mark_computed</span><span class="o">.</span><span class="n">join</span><span class="p">()</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_result</span><span class="o">.</span><span class="n">state</span></div> |
| |
| <div class="viewcode-block" id="Recording.describe"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.Recording.describe">[docs]</a> <span class="k">def</span> <span class="nf">describe</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> dict[str, int]</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns a dictionary describing the cache and recording."""</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">size</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span> |
| <span class="n">cache_manager</span><span class="o">.</span><span class="n">size</span><span class="p">(</span><span class="s1">'full'</span><span class="p">,</span> <span class="n">s</span><span class="o">.</span><span class="n">cache_key</span><span class="p">)</span> <span class="k">for</span> <span class="n">s</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_streams</span><span class="o">.</span><span class="n">values</span><span class="p">())</span> |
| <span class="k">return</span> <span class="p">{</span><span class="s1">'size'</span><span class="p">:</span> <span class="n">size</span><span class="p">,</span> <span class="s1">'duration'</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_duration_secs</span><span class="p">}</span></div></div> |
| |
| |
| <div class="viewcode-block" id="RecordingManager"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.RecordingManager">[docs]</a><span class="k">class</span> <span class="nc">RecordingManager</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""Manages recordings of PCollections for a given 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">user_pipeline</span><span class="p">,</span> <span class="n">pipeline_var</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">test_limiters</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="c1"># type: (beam.Pipeline, str, list[Limiter]) -> None # noqa: F821</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="c1"># type: beam.Pipeline</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">pipeline_var</span> <span class="o">=</span> <span class="n">pipeline_var</span> <span class="k">if</span> <span class="n">pipeline_var</span> <span class="k">else</span> <span class="s1">''</span> <span class="c1"># type: str</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_recordings</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> <span class="c1"># type: set[Recording]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_start_time_sec</span> <span class="o">=</span> <span class="mi">0</span> <span class="c1"># type: float</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_test_limiters</span> <span class="o">=</span> <span class="n">test_limiters</span> <span class="k">if</span> <span class="n">test_limiters</span> <span class="k">else</span> <span class="p">[]</span> |
| |
| <span class="k">def</span> <span class="nf">_watch</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcolls</span><span class="p">):</span> |
| <span class="c1"># type: (List[beam.pvalue.PCollection]) -> None # noqa: F821</span> |
| |
| <span class="w"> </span><span class="sd">"""Watch any pcollections not being watched.</span> |
| |
| <span class="sd"> This allows for the underlying caching layer to identify the PCollection as</span> |
| <span class="sd"> something to be cached.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">watched_pcollections</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| <span class="n">watched_dataframes</span> <span class="o">=</span> <span class="nb">set</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">_</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="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="n">watched_pcollections</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">val</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">val</span><span class="p">,</span> <span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="n">watched_dataframes</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">val</span><span class="p">)</span> |
| |
| <span class="c1"># Convert them one-by-one to generate a unique label for each. This allows</span> |
| <span class="c1"># caching at a more fine-grained granularity.</span> |
| <span class="c1">#</span> |
| <span class="c1"># TODO(https://github.com/apache/beam/issues/20929): investigate the mixing</span> |
| <span class="c1"># pcollections in multiple pipelines error when using the default label.</span> |
| <span class="k">for</span> <span class="n">df</span> <span class="ow">in</span> <span class="n">watched_dataframes</span><span class="p">:</span> |
| <span class="n">pcoll</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">utils</span><span class="o">.</span><span class="n">deferred_df_to_pcollection</span><span class="p">(</span><span class="n">df</span><span class="p">)</span> |
| <span class="n">watched_pcollections</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">for</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">pcoll</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">watched_pcollections</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">watch</span><span class="p">(</span> |
| <span class="p">{</span><span class="s1">'anonymous_pcollection_</span><span class="si">{}</span><span class="s1">'</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">id</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="k">def</span> <span class="nf">_clear</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> None</span> |
| |
| <span class="w"> </span><span class="sd">"""Clears the recording of all non-source PCollections."""</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="c1"># Only clear the PCollections that aren't being populated from the</span> |
| <span class="c1"># BackgroundCachingJob.</span> |
| <span class="n">computed</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">computed_pcollections</span> |
| <span class="n">cacheables</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">c</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">utils</span><span class="o">.</span><span class="n">cacheables</span><span class="p">()</span><span class="o">.</span><span class="n">values</span><span class="p">()</span> |
| <span class="k">if</span> <span class="n">c</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="bp">self</span><span class="o">.</span><span class="n">user_pipeline</span> <span class="ow">and</span> <span class="n">c</span><span class="o">.</span><span class="n">pcoll</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">computed</span> |
| <span class="p">]</span> |
| <span class="n">all_cached</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">c</span><span class="o">.</span><span class="n">to_key</span><span class="p">())</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">cacheables</span><span class="p">)</span> |
| <span class="n">source_pcolls</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">cache_manager</span><span class="p">,</span> <span class="s1">'capture_keys'</span><span class="p">,</span> <span class="nb">set</span><span class="p">())</span> |
| <span class="n">to_clear</span> <span class="o">=</span> <span class="n">all_cached</span> <span class="o">-</span> <span class="n">source_pcolls</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_clear_pcolls</span><span class="p">(</span><span class="n">cache_manager</span><span class="p">,</span> <span class="nb">set</span><span class="p">(</span><span class="n">to_clear</span><span class="p">))</span> |
| |
| <span class="k">def</span> <span class="nf">_clear_pcolls</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">pcolls</span><span class="p">):</span> |
| <span class="k">for</span> <span class="n">pc</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="p">:</span> |
| <span class="n">cache_manager</span><span class="o">.</span><span class="n">clear</span><span class="p">(</span><span class="s1">'full'</span><span class="p">,</span> <span class="n">pc</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="RecordingManager.clear"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.RecordingManager.clear">[docs]</a> <span class="k">def</span> <span class="nf">clear</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> None</span> |
| |
| <span class="w"> </span><span class="sd">"""Clears all cached PCollections for this RecordingManager."""</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="k">if</span> <span class="n">cache_manager</span><span class="p">:</span> |
| <span class="n">cache_manager</span><span class="o">.</span><span class="n">cleanup</span><span class="p">()</span></div> |
| |
| <div class="viewcode-block" id="RecordingManager.cancel"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.RecordingManager.cancel">[docs]</a> <span class="k">def</span> <span class="nf">cancel</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: (None) -> None</span> |
| |
| <span class="w"> </span><span class="sd">"""Cancels the current background recording job."""</span> |
| |
| <span class="n">bcj</span><span class="o">.</span><span class="n">attempt_to_cancel_background_caching_job</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="k">for</span> <span class="n">r</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_recordings</span><span class="p">:</span> |
| <span class="n">r</span><span class="o">.</span><span class="n">wait_until_finish</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_recordings</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| |
| <span class="c1"># The recordings rely on a reference to the BCJ to correctly finish. So we</span> |
| <span class="c1"># evict the BCJ after they complete.</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">evict_background_caching_job</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">user_pipeline</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="RecordingManager.describe"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.RecordingManager.describe">[docs]</a> <span class="k">def</span> <span class="nf">describe</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> dict[str, int]</span> |
| |
| <span class="w"> </span><span class="sd">"""Returns a dictionary describing the cache and recording."""</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">capture_size</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">cache_manager</span><span class="p">,</span> <span class="s1">'capture_size'</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span> |
| |
| <span class="n">descriptions</span> <span class="o">=</span> <span class="p">[</span><span class="n">r</span><span class="o">.</span><span class="n">describe</span><span class="p">()</span> <span class="k">for</span> <span class="n">r</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_recordings</span><span class="p">]</span> |
| <span class="n">size</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">d</span><span class="p">[</span><span class="s1">'size'</span><span class="p">]</span> <span class="k">for</span> <span class="n">d</span> <span class="ow">in</span> <span class="n">descriptions</span><span class="p">)</span> <span class="o">+</span> <span class="n">capture_size</span> |
| <span class="n">start</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_start_time_sec</span> |
| <span class="n">bcj</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_background_caching_job</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="k">if</span> <span class="n">bcj</span><span class="p">:</span> |
| <span class="n">state</span> <span class="o">=</span> <span class="n">bcj</span><span class="o">.</span><span class="n">state</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">state</span> <span class="o">=</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">STOPPED</span> |
| <span class="k">return</span> <span class="p">{</span> |
| <span class="s1">'size'</span><span class="p">:</span> <span class="n">size</span><span class="p">,</span> |
| <span class="s1">'start'</span><span class="p">:</span> <span class="n">start</span><span class="p">,</span> |
| <span class="s1">'state'</span><span class="p">:</span> <span class="n">state</span><span class="p">,</span> |
| <span class="s1">'pipeline_var'</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline_var</span> |
| <span class="p">}</span></div> |
| |
| <div class="viewcode-block" id="RecordingManager.record_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.RecordingManager.record_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">record_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># type: () -> bool</span> |
| |
| <span class="w"> </span><span class="sd">"""Starts a background caching job for this RecordingManager's pipeline."""</span> |
| |
| <span class="n">runner</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">runner</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">runner</span><span class="p">,</span> <span class="n">ir</span><span class="o">.</span><span class="n">InteractiveRunner</span><span class="p">):</span> |
| <span class="n">runner</span> <span class="o">=</span> <span class="n">runner</span><span class="o">.</span><span class="n">_underlying_runner</span> |
| |
| <span class="c1"># Make sure that sources without a user reference are still cached.</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_user_pipeline</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">utils</span><span class="o">.</span><span class="n">watch_sources</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="c1"># Attempt to run background caching job to record any sources.</span> |
| <span class="n">warnings</span><span class="o">.</span><span class="n">filterwarnings</span><span class="p">(</span> |
| <span class="s1">'ignore'</span><span class="p">,</span> |
| <span class="s1">'options is deprecated since First stable release. References to '</span> |
| <span class="s1">'<pipeline>.options will not be supported'</span><span class="p">,</span> |
| <span class="n">category</span><span class="o">=</span><span class="ne">DeprecationWarning</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">bcj</span><span class="o">.</span><span class="n">attempt_to_run_background_caching_job</span><span class="p">(</span> |
| <span class="n">runner</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">user_pipeline</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">user_pipeline</span><span class="o">.</span><span class="n">options</span><span class="p">,</span> |
| <span class="n">limiters</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_test_limiters</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_start_time_sec</span> <span class="o">=</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> |
| <span class="k">return</span> <span class="kc">True</span> |
| <span class="k">return</span> <span class="kc">False</span></div> |
| |
| <div class="viewcode-block" id="RecordingManager.record"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.RecordingManager.record">[docs]</a> <span class="k">def</span> <span class="nf">record</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcolls</span><span class="p">,</span> <span class="n">max_n</span><span class="p">,</span> <span class="n">max_duration</span><span class="p">):</span> |
| <span class="c1"># type: (List[beam.pvalue.PCollection], int, Union[int,str]) -> Recording # noqa: F821</span> |
| |
| <span class="w"> </span><span class="sd">"""Records the given PCollections."""</span> |
| |
| <span class="c1"># Assert that all PCollection come from the same 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="k">assert</span> <span class="n">pcoll</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="p">,</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">{}</span><span class="s1"> belongs to a different user-defined pipeline (</span><span class="si">{}</span><span class="s1">) than that of'</span> |
| <span class="s1">' other PCollections (</span><span class="si">{}</span><span class="s1">).'</span><span class="o">.</span><span class="n">format</span><span class="p">(</span> |
| <span class="n">pcoll</span><span class="p">,</span> <span class="n">pcoll</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">user_pipeline</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">max_duration</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="ow">and</span> <span class="n">max_duration</span> <span class="o">!=</span> <span class="s1">'inf'</span><span class="p">:</span> |
| <span class="n">max_duration_secs</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">to_timedelta</span><span class="p">(</span><span class="n">max_duration</span><span class="p">)</span><span class="o">.</span><span class="n">total_seconds</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">max_duration_secs</span> <span class="o">=</span> <span class="n">max_duration</span> |
| |
| <span class="c1"># Make sure that all PCollections to be shown are watched. If a PCollection</span> |
| <span class="c1"># has not been watched, make up a variable name for that PCollection and</span> |
| <span class="c1"># watch it. No validation is needed here because the watch logic can handle</span> |
| <span class="c1"># arbitrary variables.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_watch</span><span class="p">(</span><span class="n">pcolls</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">record_pipeline</span><span class="p">()</span> |
| |
| <span class="c1"># Get the subset of computed PCollections. These do not to be recomputed.</span> |
| <span class="n">computed_pcolls</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span> |
| <span class="n">pcoll</span> <span class="k">for</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls</span> |
| <span class="k">if</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="c1"># Start a pipeline fragment to start computing the PCollections.</span> |
| <span class="n">uncomputed_pcolls</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">pcolls</span><span class="p">)</span><span class="o">.</span><span class="n">difference</span><span class="p">(</span><span class="n">computed_pcolls</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">uncomputed_pcolls</span><span class="p">:</span> |
| <span class="c1"># Clear the cache of the given uncomputed PCollections because they are</span> |
| <span class="c1"># incomplete.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_clear</span><span class="p">()</span> |
| |
| <span class="n">warnings</span><span class="o">.</span><span class="n">filterwarnings</span><span class="p">(</span> |
| <span class="s1">'ignore'</span><span class="p">,</span> |
| <span class="s1">'options is deprecated since First stable release. References to '</span> |
| <span class="s1">'<pipeline>.options will not be supported'</span><span class="p">,</span> |
| <span class="n">category</span><span class="o">=</span><span class="ne">DeprecationWarning</span><span class="p">)</span> |
| <span class="n">cache_path</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">options</span><span class="o">.</span><span class="n">cache_root</span> |
| <span class="n">is_remote_run</span> <span class="o">=</span> <span class="n">cache_path</span> <span class="ow">and</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">(</span> |
| <span class="p">)</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">cache_root</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">'gs://'</span><span class="p">)</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">uncomputed_pcolls</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">options</span><span class="p">)</span><span class="o">.</span><span class="n">run</span><span class="p">(</span><span class="n">blocking</span><span class="o">=</span><span class="n">is_remote_run</span><span class="p">)</span> |
| <span class="n">result</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">pipeline_result</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="k">else</span><span class="p">:</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="n">recording</span> <span class="o">=</span> <span class="n">Recording</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">pcolls</span><span class="p">,</span> <span class="n">result</span><span class="p">,</span> <span class="n">max_n</span><span class="p">,</span> <span class="n">max_duration_secs</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_recordings</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">recording</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">recording</span></div> |
| |
| <div class="viewcode-block" id="RecordingManager.read"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.recording_manager.html#apache_beam.runners.interactive.recording_manager.RecordingManager.read">[docs]</a> <span class="k">def</span> <span class="nf">read</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll_name</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">,</span> <span class="n">max_n</span><span class="p">,</span> <span class="n">max_duration_secs</span><span class="p">):</span> |
| <span class="c1"># type: (str, beam.pvalue.PValue, int, float) -> Union[None, ElementStream] # noqa: F821</span> |
| |
| <span class="w"> </span><span class="sd">"""Reads an ElementStream of a computed PCollection.</span> |
| |
| <span class="sd"> Returns None if an error occurs. The caller is responsible of validating if</span> |
| <span class="sd"> the given pcoll_name and pcoll can identify a watched and computed</span> |
| <span class="sd"> PCollection without ambiguity in the notebook.</span> |
| <span class="sd"> """</span> |
| |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">cache_key</span> <span class="o">=</span> <span class="n">CacheKey</span><span class="o">.</span><span class="n">from_pcoll</span><span class="p">(</span><span class="n">pcoll_name</span><span class="p">,</span> <span class="n">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="n">ElementStream</span><span class="p">(</span> |
| <span class="n">pcoll</span><span class="p">,</span> <span class="n">pcoll_name</span><span class="p">,</span> <span class="n">cache_key</span><span class="p">,</span> <span class="n">max_n</span><span class="p">,</span> <span class="n">max_duration_secs</span><span class="p">)</span> |
| <span class="k">except</span> <span class="p">(</span><span class="ne">KeyboardInterrupt</span><span class="p">,</span> <span class="ne">SystemExit</span><span class="p">):</span> |
| <span class="k">raise</span> |
| <span class="k">except</span> <span class="ne">Exception</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span> |
| <span class="c1"># Caller should handle all validations. Here to avoid redundant</span> |
| <span class="c1"># validations, simply log errors if caller fails to do so.</span> |
| <span class="n">_LOGGER</span><span class="o">.</span><span class="n">error</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">e</span><span class="p">))</span> |
| <span class="k">return</span> <span class="kc">None</span></div></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> |