blob: f1a6eeb7799daefa90fa39742d2d34b7a36d29e3 [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.runners.interactive.interactive_environment &mdash; Apache Beam 2.47.0 documentation</title>
<script type="text/javascript" src="../../../../_static/js/modernizr.min.js"></script>
<script type="text/javascript" id="documentation_options" data-url_root="../../../../" src="../../../../_static/documentation_options.js"></script>
<script type="text/javascript" src="../../../../_static/jquery.js"></script>
<script type="text/javascript" src="../../../../_static/underscore.js"></script>
<script type="text/javascript" src="../../../../_static/doctools.js"></script>
<script type="text/javascript" src="../../../../_static/language_data.js"></script>
<script async="async" type="text/javascript" src="https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.5/latest.js?config=TeX-AMS-MML_HTMLorMML"></script>
<script type="text/javascript" src="../../../../_static/js/theme.js"></script>
<link rel="stylesheet" href="../../../../_static/css/theme.css" type="text/css" />
<link rel="stylesheet" href="../../../../_static/pygments.css" type="text/css" />
<link rel="index" title="Index" href="../../../../genindex.html" />
<link rel="search" title="Search" href="../../../../search.html" />
</head>
<body class="wy-body-for-nav">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search" >
<a href="../../../../index.html" class="icon icon-home"> Apache Beam
</a>
<div class="version">
2.47.0
</div>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.dataframe.html">apache_beam.dataframe package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.ml.html">apache_beam.ml package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.utils.html">apache_beam.utils package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.yaml.html">apache_beam.yaml package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../../index.html">Docs</a> &raquo;</li>
<li><a href="../../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.runners.interactive.interactive_environment</li>
<li class="wy-breadcrumbs-aside">
</li>
</ul>
<hr/>
</div>
<div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article">
<div itemprop="articleBody">
<h1>Source code for apache_beam.runners.interactive.interactive_environment</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="sd">&quot;&quot;&quot;Module of the current Interactive Beam environment.</span>
<span class="sd">For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd">Provides interfaces to interact with existing Interactive Beam environment.</span>
<span class="sd">External Interactive Beam users please use interactive_beam module in</span>
<span class="sd">application code or notebook.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span> <span class="nn">atexit</span>
<span class="kn">import</span> <span class="nn">importlib</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">import</span> <span class="nn">os</span>
<span class="kn">import</span> <span class="nn">tempfile</span>
<span class="kn">import</span> <span class="nn">warnings</span>
<span class="kn">from</span> <span class="nn">collections.abc</span> <span class="kn">import</span> <span class="n">Iterable</span>
<span class="kn">from</span> <span class="nn">pathlib</span> <span class="kn">import</span> <span class="n">PurePath</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.runners</span> <span class="kn">import</span> <span class="n">DataflowRunner</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners</span> <span class="kn">import</span> <span class="n">runner</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.direct</span> <span class="kn">import</span> <span class="n">direct_runner</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="kn">import</span> <span class="n">cache_manager</span> <span class="k">as</span> <span class="n">cache</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.messaging.interactive_environment_inspector</span> <span class="kn">import</span> <span class="n">InteractiveEnvironmentInspector</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.recording_manager</span> <span class="kn">import</span> <span class="n">RecordingManager</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.sql.sql_chain</span> <span class="kn">import</span> <span class="n">SqlChain</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.user_pipeline_tracker</span> <span class="kn">import</span> <span class="n">UserPipelineTracker</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.utils</span> <span class="kn">import</span> <span class="n">assert_bucket_exists</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.utils</span> <span class="kn">import</span> <span class="n">detect_pipeline_runner</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.utils</span> <span class="kn">import</span> <span class="n">register_ipython_log_handler</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.interactive_utils</span> <span class="kn">import</span> <span class="n">is_in_ipython</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.interactive_utils</span> <span class="kn">import</span> <span class="n">is_in_notebook</span>
<span class="c1"># Interactive Beam user flow is data-centric rather than pipeline-centric, so</span>
<span class="c1"># there is only one global interactive environment instance that manages</span>
<span class="c1"># implementation that enables interactivity.</span>
<span class="n">_interactive_beam_env</span> <span class="o">=</span> <span class="kc">None</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>
<span class="c1"># By `format(customized_script=xxx)`, the given `customized_script` is</span>
<span class="c1"># guaranteed to be executed within access to a jquery with datatable plugin</span>
<span class="c1"># configured which is useful so that any `customized_script` is resilient to</span>
<span class="c1"># browser refresh. Inside `customized_script`, use `$` as jQuery.</span>
<span class="n">_JQUERY_WITH_DATATABLE_TEMPLATE</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> if (typeof window.interactive_beam_jquery == &#39;undefined&#39;) {{</span>
<span class="s2"> var jqueryScript = document.createElement(&#39;script&#39;);</span>
<span class="s2"> jqueryScript.src = &#39;https://code.jquery.com/jquery-3.4.1.slim.min.js&#39;;</span>
<span class="s2"> jqueryScript.type = &#39;text/javascript&#39;;</span>
<span class="s2"> jqueryScript.onload = function() {{</span>
<span class="s2"> var datatableScript = document.createElement(&#39;script&#39;);</span>
<span class="s2"> datatableScript.src = &#39;https://cdn.datatables.net/1.10.20/js/jquery.dataTables.min.js&#39;;</span>
<span class="s2"> datatableScript.type = &#39;text/javascript&#39;;</span>
<span class="s2"> datatableScript.onload = function() {{</span>
<span class="s2"> window.interactive_beam_jquery = jQuery.noConflict(true);</span>
<span class="s2"> window.interactive_beam_jquery(document).ready(function($){{</span>
<span class="s2"> </span><span class="si">{customized_script}</span>
<span class="s2"> }});</span>
<span class="s2"> }}</span>
<span class="s2"> document.head.appendChild(datatableScript);</span>
<span class="s2"> }};</span>
<span class="s2"> document.head.appendChild(jqueryScript);</span>
<span class="s2"> }} else {{</span>
<span class="s2"> window.interactive_beam_jquery(document).ready(function($){{</span>
<span class="s2"> </span><span class="si">{customized_script}</span>
<span class="s2"> }});</span>
<span class="s2"> }}&quot;&quot;&quot;</span>
<span class="c1"># By `format(hrefs=xxx)`, the given `hrefs` will be imported as HTML imports.</span>
<span class="c1"># Since HTML import might not be supported by the browser, we check if HTML</span>
<span class="c1"># import is supported by the browser, if so, import HTMLs else setup</span>
<span class="c1"># webcomponents and chain the HTML import to the end of onload.</span>
<span class="n">_HTML_IMPORT_TEMPLATE</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> var import_html = () =&gt; {{</span>
<span class="s2"> </span><span class="si">{hrefs}</span><span class="s2">.forEach(href =&gt; {{</span>
<span class="s2"> var link = document.createElement(&#39;link&#39;);</span>
<span class="s2"> link.rel = &#39;import&#39;</span>
<span class="s2"> link.href = href;</span>
<span class="s2"> document.head.appendChild(link);</span>
<span class="s2"> }});</span>
<span class="s2"> }}</span>
<span class="s2"> if (&#39;import&#39; in document.createElement(&#39;link&#39;)) {{</span>
<span class="s2"> import_html();</span>
<span class="s2"> }} else {{</span>
<span class="s2"> var webcomponentScript = document.createElement(&#39;script&#39;);</span>
<span class="s2"> webcomponentScript.src = &#39;https://cdnjs.cloudflare.com/ajax/libs/webcomponentsjs/1.3.3/webcomponents-lite.js&#39;;</span>
<span class="s2"> webcomponentScript.type = &#39;text/javascript&#39;;</span>
<span class="s2"> webcomponentScript.onload = function(){{</span>
<span class="s2"> import_html();</span>
<span class="s2"> }};</span>
<span class="s2"> document.head.appendChild(webcomponentScript);</span>
<span class="s2"> }}&quot;&quot;&quot;</span>
<div class="viewcode-block" id="current_env"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.current_env">[docs]</a><span class="k">def</span> <span class="nf">current_env</span><span class="p">():</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets current Interactive Beam environment.&quot;&quot;&quot;</span>
<span class="k">global</span> <span class="n">_interactive_beam_env</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">_interactive_beam_env</span><span class="p">:</span>
<span class="n">_interactive_beam_env</span> <span class="o">=</span> <span class="n">InteractiveEnvironment</span><span class="p">()</span>
<span class="k">return</span> <span class="n">_interactive_beam_env</span></div>
<div class="viewcode-block" id="new_env"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.new_env">[docs]</a><span class="k">def</span> <span class="nf">new_env</span><span class="p">():</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Creates a new Interactive Beam environment to replace current one.&quot;&quot;&quot;</span>
<span class="k">global</span> <span class="n">_interactive_beam_env</span>
<span class="k">if</span> <span class="n">_interactive_beam_env</span><span class="p">:</span>
<span class="n">_interactive_beam_env</span><span class="o">.</span><span class="n">cleanup</span><span class="p">()</span>
<span class="n">_interactive_beam_env</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">return</span> <span class="n">current_env</span><span class="p">()</span></div>
<div class="viewcode-block" id="InteractiveEnvironment"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment">[docs]</a><span class="k">class</span> <span class="nc">InteractiveEnvironment</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;An interactive environment with cache and pipeline variable metadata.</span>
<span class="sd"> Interactive Beam will use the watched variable information to determine if a</span>
<span class="sd"> PCollection is assigned to a variable in user pipeline definition. When</span>
<span class="sd"> executing the pipeline, interactivity is applied with implicit cache</span>
<span class="sd"> mechanism for those PCollections if the pipeline is interactive. Users can</span>
<span class="sd"> also visualize and introspect those PCollections in user code since they have</span>
<span class="sd"> handles to the variables.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># Registers a cleanup routine when system exits.</span>
<span class="n">atexit</span><span class="o">.</span><span class="n">register</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">cleanup</span><span class="p">)</span>
<span class="c1"># Holds cache managers that manage source recording and intermediate</span>
<span class="c1"># PCollection cache for each pipeline. Each key is a stringified user</span>
<span class="c1"># defined pipeline instance&#39;s id.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_managers</span> <span class="o">=</span> <span class="p">{}</span>
<span class="c1"># Holds RecordingManagers keyed by pipeline instance id.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</span> <span class="o">=</span> <span class="p">{}</span>
<span class="c1"># Holds class instances, module object, string of module names.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_watching_set</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="c1"># Holds variables list of (Dict[str, object]).</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_watching_dict_list</span> <span class="o">=</span> <span class="p">[]</span>
<span class="c1"># Holds results of main jobs as Dict[str, PipelineResult].</span>
<span class="c1"># Each key is a pipeline instance defined by the end user. The</span>
<span class="c1"># InteractiveRunner is responsible for populating this dictionary</span>
<span class="c1"># implicitly.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_main_pipeline_results</span> <span class="o">=</span> <span class="p">{}</span>
<span class="c1"># Holds background caching jobs as Dict[str, BackgroundCachingJob].</span>
<span class="c1"># Each key is a pipeline instance defined by the end user. The</span>
<span class="c1"># InteractiveRunner or its enclosing scope is responsible for populating</span>
<span class="c1"># this dictionary implicitly when a background caching jobs is started.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_jobs</span> <span class="o">=</span> <span class="p">{}</span>
<span class="c1"># Holds TestStreamServiceControllers that controls gRPC servers serving</span>
<span class="c1"># events as test stream of TestStreamPayload.Event.</span>
<span class="c1"># Dict[str, TestStreamServiceController]. Each key is a pipeline</span>
<span class="c1"># instance defined by the end user. The InteractiveRunner or its enclosing</span>
<span class="c1"># scope is responsible for populating this dictionary implicitly when a new</span>
<span class="c1"># controller is created to start a new gRPC server. The server stays alive</span>
<span class="c1"># until a new background caching job is started thus invalidating everything</span>
<span class="c1"># the gRPC server serves.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_test_stream_service_controllers</span> <span class="o">=</span> <span class="p">{}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cached_source_signature</span> <span class="o">=</span> <span class="p">{}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_tracked_user_pipelines</span> <span class="o">=</span> <span class="n">UserPipelineTracker</span><span class="p">()</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.interactive_beam</span> <span class="kn">import</span> <span class="n">clusters</span>
<span class="bp">self</span><span class="o">.</span><span class="n">clusters</span> <span class="o">=</span> <span class="n">clusters</span>
<span class="c1"># Tracks the computation completeness of PCollections. PCollections tracked</span>
<span class="c1"># here don&#39;t need to be re-computed when data introspection is needed.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_computed_pcolls</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="c1"># Always watch __main__ module.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">watch</span><span class="p">(</span><span class="s1">&#39;__main__&#39;</span><span class="p">)</span>
<span class="c1"># Check if [interactive] dependencies are installed.</span>
<span class="k">try</span><span class="p">:</span>
<span class="kn">import</span> <span class="nn">IPython</span> <span class="c1"># pylint: disable=unused-import</span>
<span class="kn">import</span> <span class="nn">timeloop</span> <span class="c1"># pylint: disable=unused-import</span>
<span class="kn">from</span> <span class="nn">facets_overview.generic_feature_statistics_generator</span> <span class="kn">import</span> <span class="n">GenericFeatureStatisticsGenerator</span> <span class="c1"># pylint: disable=unused-import</span>
<span class="kn">from</span> <span class="nn">google.cloud</span> <span class="kn">import</span> <span class="n">dataproc_v1</span> <span class="c1"># pylint: disable=unused-import</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_is_interactive_ready</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_is_interactive_ready</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;Dependencies required for Interactive Beam PCollection &#39;</span>
<span class="s1">&#39;visualization are not available, please use: `pip &#39;</span>
<span class="s1">&#39;install apache-beam[interactive]` to install necessary &#39;</span>
<span class="s1">&#39;dependencies to enable all data visualization features.&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_is_in_ipython</span> <span class="o">=</span> <span class="n">is_in_ipython</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_is_in_notebook</span> <span class="o">=</span> <span class="n">is_in_notebook</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">_is_in_ipython</span><span class="p">:</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;You cannot use Interactive Beam features when you are &#39;</span>
<span class="s1">&#39;not in an interactive environment such as a Jupyter &#39;</span>
<span class="s1">&#39;notebook or ipython terminal.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_in_ipython</span> <span class="ow">and</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_in_notebook</span><span class="p">:</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;You have limited Interactive Beam features since your &#39;</span>
<span class="s1">&#39;ipython kernel is not connected to any notebook frontend.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_in_notebook</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">load_jquery_with_datatable</span><span class="p">()</span>
<span class="n">register_ipython_log_handler</span><span class="p">()</span>
<span class="c1"># A singleton inspector instance to message information of current</span>
<span class="c1"># environment to other applications.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_inspector</span> <span class="o">=</span> <span class="n">InteractiveEnvironmentInspector</span><span class="p">()</span>
<span class="c1"># A similar singleton inspector except it includes synthetic variables</span>
<span class="c1"># generated by Interactive Beam.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_inspector_with_synthetic</span> <span class="o">=</span> <span class="n">InteractiveEnvironmentInspector</span><span class="p">(</span>
<span class="n">ignore_synthetic</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">sql_chain</span> <span class="o">=</span> <span class="p">{}</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">options</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A reference to the global interactive options.</span>
<span class="sd"> Provided to avoid import loop or excessive dynamic import. All internal</span>
<span class="sd"> Interactive Beam modules should access interactive_beam.options through</span>
<span class="sd"> this property.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.interactive_beam</span> <span class="kn">import</span> <span class="n">options</span>
<span class="k">return</span> <span class="n">options</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">is_interactive_ready</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;If the [interactive] dependencies are installed.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_interactive_ready</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">is_in_ipython</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;If the runtime is within an IPython kernel.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_in_ipython</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">is_in_notebook</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;If the kernel is connected to a notebook frontend.</span>
<span class="sd"> If not, it could be that the user is using kernel in a terminal or a unit</span>
<span class="sd"> test.</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">_is_in_notebook</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">inspector</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets the singleton InteractiveEnvironmentInspector to retrieve</span>
<span class="sd"> information consumable by other applications such as a notebook</span>
<span class="sd"> extension.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_inspector</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">inspector_with_synthetic</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets the singleton InteractiveEnvironmentInspector with additional</span>
<span class="sd"> synthetic variables generated by Interactive Beam. Internally used.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_inspector_with_synthetic</span>
<div class="viewcode-block" id="InteractiveEnvironment.cleanup_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.cleanup_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">cleanup_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="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="n">bcj</span><span class="o">.</span><span class="n">attempt_to_cancel_background_caching_job</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="n">bcj</span><span class="o">.</span><span class="n">attempt_to_stop_test_stream_service</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="n">cache_manager</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="c1"># Recording manager performs cache manager cleanup during eviction, so we</span>
<span class="c1"># don&#39;t need to clean it up here.</span>
<span class="k">if</span> <span class="n">cache_manager</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_recording_manager</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">cache_manager</span><span class="o">.</span><span class="n">cleanup</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">clusters</span><span class="o">.</span><span class="n">cleanup</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.cleanup_environment"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.cleanup_environment">[docs]</a> <span class="k">def</span> <span class="nf">cleanup_environment</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">job</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_jobs</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">if</span> <span class="n">job</span><span class="p">:</span>
<span class="n">job</span><span class="o">.</span><span class="n">cancel</span><span class="p">()</span>
<span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">controller</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_test_stream_service_controllers</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">if</span> <span class="n">controller</span><span class="p">:</span>
<span class="n">controller</span><span class="o">.</span><span class="n">stop</span><span class="p">()</span>
<span class="k">for</span> <span class="n">pipeline_id</span><span class="p">,</span> <span class="n">cache_manager</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_managers</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="c1"># Recording manager performs cache manager cleanup during eviction, so</span>
<span class="c1"># we don&#39;t need to clean it up here.</span>
<span class="k">if</span> <span class="n">cache_manager</span> <span class="ow">and</span> <span class="n">pipeline_id</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</span><span class="p">:</span>
<span class="n">cache_manager</span><span class="o">.</span><span class="n">cleanup</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">clusters</span><span class="o">.</span><span class="n">cleanup</span><span class="p">(</span><span class="n">force</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.cleanup"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.cleanup">[docs]</a> <span class="k">def</span> <span class="nf">cleanup</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Cleans up cached states for the given pipeline. Noop if the given</span>
<span class="sd"> pipeline is absent from the environment. Cleans up for all pipelines</span>
<span class="sd"> if no pipeline is specified.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">cleanup_pipeline</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">cleanup_environment</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">evict_recording_manager</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">evict_background_caching_job</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">evict_test_stream_service_controller</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">evict_computed_pcollections</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">evict_cached_source_signature</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">evict_pipeline_result</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">evict_tracked_pipelines</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_track_user_pipelines</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">watchable</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Tracks user pipelines from the given watchable.&quot;&quot;&quot;</span>
<span class="n">pipelines</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">watchable</span><span class="p">,</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">):</span>
<span class="n">pipelines</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">watchable</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">watchable</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">watchable</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">):</span>
<span class="n">pipelines</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">v</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">watchable</span><span class="p">,</span> <span class="n">Iterable</span><span class="p">):</span>
<span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">watchable</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">):</span>
<span class="n">pipelines</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">v</span><span class="p">)</span>
<span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="n">pipelines</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_tracked_user_pipelines</span><span class="o">.</span><span class="n">add_user_pipeline</span><span class="p">(</span><span class="n">p</span><span class="p">)</span>
<span class="n">_</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span><span class="n">p</span><span class="p">,</span> <span class="n">create_if_absent</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">_</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_recording_manager</span><span class="p">(</span><span class="n">p</span><span class="p">,</span> <span class="n">create_if_absent</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<div class="viewcode-block" id="InteractiveEnvironment.watch"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.watch">[docs]</a> <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">watchable</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Watches a watchable.</span>
<span class="sd"> A watchable can be a dictionary of variable metadata such as locals(), a str</span>
<span class="sd"> name of a module, a module object or an instance of a class. The variable</span>
<span class="sd"> can come from any scope even local. Duplicated variable naming doesn&#39;t</span>
<span class="sd"> matter since they are different instances. Duplicated variables are also</span>
<span class="sd"> allowed when watching.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">watchable</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_watching_dict_list</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">watchable</span><span class="o">.</span><span class="n">items</span><span class="p">())</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_watching_set</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">watchable</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_track_user_pipelines</span><span class="p">(</span><span class="n">watchable</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.watching"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.watching">[docs]</a> <span class="k">def</span> <span class="nf">watching</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Analyzes and returns a list of pair lists referring to variable names and</span>
<span class="sd"> values from watched scopes.</span>
<span class="sd"> Each entry in the list represents the variable defined within a watched</span>
<span class="sd"> watchable. Currently, each entry holds a list of pairs. The format might</span>
<span class="sd"> change in the future to hold more metadata. Duplicated pairs are allowed.</span>
<span class="sd"> And multiple paris can have the same variable name as the &quot;first&quot; while</span>
<span class="sd"> having different variable values as the &quot;second&quot; since variables in</span>
<span class="sd"> different scopes can have the same name.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">watching</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_watching_dict_list</span><span class="p">)</span>
<span class="k">for</span> <span class="n">watchable</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_watching_set</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">watchable</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span>
<span class="n">module</span> <span class="o">=</span> <span class="n">importlib</span><span class="o">.</span><span class="n">import_module</span><span class="p">(</span><span class="n">watchable</span><span class="p">)</span>
<span class="n">watching</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="nb">vars</span><span class="p">(</span><span class="n">module</span><span class="p">)</span><span class="o">.</span><span class="n">items</span><span class="p">())</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">watching</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="nb">vars</span><span class="p">(</span><span class="n">watchable</span><span class="p">)</span><span class="o">.</span><span class="n">items</span><span class="p">())</span>
<span class="k">return</span> <span class="n">watching</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.set_cache_manager"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.set_cache_manager">[docs]</a> <span class="k">def</span> <span class="nf">set_cache_manager</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</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Sets the cache manager held by current Interactive Environment for the</span>
<span class="sd"> given pipeline.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span> <span class="ow">is</span> <span class="n">cache_manager</span><span class="p">:</span>
<span class="c1"># NOOP if setting to the same cache_manager.</span>
<span class="k">return</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span><span class="n">pipeline</span><span class="p">):</span>
<span class="c1"># Invoke cleanup routine when a new cache_manager is forcefully set and</span>
<span class="c1"># current cache_manager is not None.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">cleanup</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_managers</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">pipeline</span><span class="p">))]</span> <span class="o">=</span> <span class="n">cache_manager</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.get_cache_manager"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.get_cache_manager">[docs]</a> <span class="k">def</span> <span class="nf">get_cache_manager</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">create_if_absent</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets the cache manager held by current Interactive Environment for the</span>
<span class="sd"> given pipeline. If the pipeline is absent from the environment while</span>
<span class="sd"> create_if_absent is True, creates and returns a new file based cache</span>
<span class="sd"> manager for the pipeline.&quot;&quot;&quot;</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">filterwarnings</span><span class="p">(</span>
<span class="s1">&#39;ignore&#39;</span><span class="p">,</span>
<span class="s1">&#39;options is deprecated since First stable release. References to &#39;</span>
<span class="s1">&#39;&lt;pipeline&gt;.options will not be supported&#39;</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_manager</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_managers</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)),</span> <span class="kc">None</span><span class="p">)</span>
<span class="n">pipeline_runner</span> <span class="o">=</span> <span class="n">detect_pipeline_runner</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">cache_manager</span> <span class="ow">and</span> <span class="n">create_if_absent</span><span class="p">:</span>
<span class="n">cache_root</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">cache_root</span>
<span class="k">if</span> <span class="n">cache_root</span><span class="p">:</span>
<span class="k">if</span> <span class="n">cache_root</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">&#39;gs://&#39;</span><span class="p">):</span>
<span class="n">cache_dir</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_gcs_cache_dir</span><span class="p">(</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">cache_root</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">cache_dir</span> <span class="o">=</span> <span class="n">tempfile</span><span class="o">.</span><span class="n">mkdtemp</span><span class="p">(</span><span class="nb">dir</span><span class="o">=</span><span class="n">cache_root</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pipeline_runner</span><span class="p">,</span> <span class="n">direct_runner</span><span class="o">.</span><span class="n">DirectRunner</span><span class="p">):</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span>
<span class="s1">&#39;A local cache directory has been specified while &#39;</span>
<span class="s1">&#39;not using DirectRunner. It is recommended to cache into a &#39;</span>
<span class="s1">&#39;GCS bucket instead.&#39;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">staging_location</span> <span class="o">=</span> <span class="n">pipeline</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">get_all_options</span><span class="p">(</span>
<span class="p">)[</span><span class="s1">&#39;staging_location&#39;</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pipeline_runner</span><span class="p">,</span> <span class="n">DataflowRunner</span><span class="p">)</span> <span class="ow">and</span> <span class="n">staging_location</span><span class="p">:</span>
<span class="n">cache_dir</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_gcs_cache_dir</span><span class="p">(</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">staging_location</span><span class="p">)</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">info</span><span class="p">(</span>
<span class="s1">&#39;No cache_root detected. &#39;</span>
<span class="s1">&#39;Defaulting to staging_location </span><span class="si">%s</span><span class="s1"> for cache location.&#39;</span><span class="p">,</span>
<span class="n">staging_location</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">cache_dir</span> <span class="o">=</span> <span class="n">tempfile</span><span class="o">.</span><span class="n">mkdtemp</span><span class="p">(</span>
<span class="n">suffix</span><span class="o">=</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)),</span>
<span class="n">prefix</span><span class="o">=</span><span class="s1">&#39;it-&#39;</span><span class="p">,</span>
<span class="nb">dir</span><span class="o">=</span><span class="n">os</span><span class="o">.</span><span class="n">environ</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;TEST_TMPDIR&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span>
<span class="n">cache_manager</span> <span class="o">=</span> <span class="n">cache</span><span class="o">.</span><span class="n">FileBasedCacheManager</span><span class="p">(</span><span class="n">cache_dir</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cache_managers</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">pipeline</span><span class="p">))]</span> <span class="o">=</span> <span class="n">cache_manager</span>
<span class="k">return</span> <span class="n">cache_manager</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.evict_cache_manager"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.evict_cache_manager">[docs]</a> <span class="k">def</span> <span class="nf">evict_cache_manager</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Evicts the cache manager held by current Interactive Environment for the</span>
<span class="sd"> given pipeline. Noop if the pipeline is absent from the environment. If no</span>
<span class="sd"> pipeline is specified, evicts for all pipelines.&quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">cleanup</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="k">if</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache_managers</span><span class="o">.</span><span class="n">pop</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">pipeline</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">_cache_managers</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.set_recording_manager"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.set_recording_manager">[docs]</a> <span class="k">def</span> <span class="nf">set_recording_manager</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">recording_manager</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Sets the recording manager for the given pipeline.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_recording_manager</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span> <span class="ow">is</span> <span class="n">recording_manager</span><span class="p">:</span>
<span class="c1"># NOOP if setting to the same recording_manager.</span>
<span class="k">return</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</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">pipeline</span><span class="p">))]</span> <span class="o">=</span> <span class="n">recording_manager</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.get_recording_manager"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.get_recording_manager">[docs]</a> <span class="k">def</span> <span class="nf">get_recording_manager</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">create_if_absent</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets the recording manager for the given pipeline.&quot;&quot;&quot;</span>
<span class="n">recording_manager</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)),</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">recording_manager</span> <span class="ow">and</span> <span class="n">create_if_absent</span><span class="p">:</span>
<span class="c1"># Get the pipeline variable name for the user. This is useful if the user</span>
<span class="c1"># has multiple pipelines.</span>
<span class="n">pipeline_var</span> <span class="o">=</span> <span class="s1">&#39;&#39;</span>
<span class="k">for</span> <span class="n">w</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">watching</span><span class="p">():</span>
<span class="k">for</span> <span class="n">var</span><span class="p">,</span> <span class="n">val</span> <span class="ow">in</span> <span class="n">w</span><span class="p">:</span>
<span class="k">if</span> <span class="n">val</span> <span class="ow">is</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="n">pipeline_var</span> <span class="o">=</span> <span class="n">var</span>
<span class="k">break</span>
<span class="n">recording_manager</span> <span class="o">=</span> <span class="n">RecordingManager</span><span class="p">(</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">pipeline_var</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</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">pipeline</span><span class="p">))]</span> <span class="o">=</span> <span class="n">recording_manager</span>
<span class="k">return</span> <span class="n">recording_manager</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.evict_recording_manager"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.evict_recording_manager">[docs]</a> <span class="k">def</span> <span class="nf">evict_recording_manager</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Evicts the recording manager for the given pipeline.</span>
<span class="sd"> This stops the background caching job and clears the cache.</span>
<span class="sd"> Noop if the pipeline is absent from the environment. If no</span>
<span class="sd"> pipeline is specified, evicts for all pipelines.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="k">for</span> <span class="n">rm</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="n">rm</span><span class="o">.</span><span class="n">cancel</span><span class="p">()</span>
<span class="n">rm</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">return</span>
<span class="n">recording_manager</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_recording_manager</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="k">if</span> <span class="n">recording_manager</span><span class="p">:</span>
<span class="n">recording_manager</span><span class="o">.</span><span class="n">cancel</span><span class="p">()</span>
<span class="n">recording_manager</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</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">pipeline</span><span class="p">))]</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.describe_all_recordings"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.describe_all_recordings">[docs]</a> <span class="k">def</span> <span class="nf">describe_all_recordings</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns a description of the recording for all watched pipelnes.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="p">{</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pipeline_id_to_pipeline</span><span class="p">(</span><span class="n">pid</span><span class="p">):</span> <span class="n">rm</span><span class="o">.</span><span class="n">describe</span><span class="p">()</span>
<span class="k">for</span> <span class="n">pid</span><span class="p">,</span>
<span class="n">rm</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_recording_managers</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.set_pipeline_result"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.set_pipeline_result">[docs]</a> <span class="k">def</span> <span class="nf">set_pipeline_result</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">result</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Sets the pipeline run result. Adds one if absent. Otherwise, replace.&quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="nb">issubclass</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">pipeline</span><span class="p">),</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">),</span> <span class="p">(</span>
<span class="s1">&#39;pipeline must be an instance of apache_beam.Pipeline or its subclass&#39;</span><span class="p">)</span>
<span class="k">assert</span> <span class="nb">issubclass</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">result</span><span class="p">),</span> <span class="n">runner</span><span class="o">.</span><span class="n">PipelineResult</span><span class="p">),</span> <span class="p">(</span>
<span class="s1">&#39;result must be an instance of &#39;</span>
<span class="s1">&#39;apache_beam.runners.runner.PipelineResult or its subclass&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_main_pipeline_results</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">pipeline</span><span class="p">))]</span> <span class="o">=</span> <span class="n">result</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.evict_pipeline_result"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.evict_pipeline_result">[docs]</a> <span class="k">def</span> <span class="nf">evict_pipeline_result</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Evicts the last run result of the given pipeline. Noop if the pipeline</span>
<span class="sd"> is absent from the environment. If no pipeline is specified, evicts for all</span>
<span class="sd"> pipelines.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_main_pipeline_results</span><span class="o">.</span><span class="n">pop</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">pipeline</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">_main_pipeline_results</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.pipeline_result"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.pipeline_result">[docs]</a> <span class="k">def</span> <span class="nf">pipeline_result</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets the pipeline run result. None if absent.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_main_pipeline_results</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)),</span> <span class="kc">None</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.set_background_caching_job"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.set_background_caching_job">[docs]</a> <span class="k">def</span> <span class="nf">set_background_caching_job</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">background_caching_job</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Sets the background caching job started from the given pipeline.&quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="nb">issubclass</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">pipeline</span><span class="p">),</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">),</span> <span class="p">(</span>
<span class="s1">&#39;pipeline must be an instance of apache_beam.Pipeline or its subclass&#39;</span><span class="p">)</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.background_caching_job</span> <span class="kn">import</span> <span class="n">BackgroundCachingJob</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">background_caching_job</span><span class="p">,</span> <span class="n">BackgroundCachingJob</span><span class="p">),</span> <span class="p">(</span>
<span class="s1">&#39;background_caching job must be an instance of BackgroundCachingJob&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_jobs</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">pipeline</span><span class="p">))]</span> <span class="o">=</span> <span class="n">background_caching_job</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.get_background_caching_job"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.get_background_caching_job">[docs]</a> <span class="k">def</span> <span class="nf">get_background_caching_job</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets the background caching job started from the given pipeline.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_jobs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)),</span> <span class="kc">None</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.evict_background_caching_job"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.evict_background_caching_job">[docs]</a> <span class="k">def</span> <span class="nf">evict_background_caching_job</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Evicts the background caching job started from the given pipeline. Noop</span>
<span class="sd"> if the given pipeline is absent from the environment. If no pipeline is</span>
<span class="sd"> specified, evicts for all pipelines.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_background_caching_jobs</span><span class="o">.</span><span class="n">pop</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">pipeline</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">_background_caching_jobs</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.set_test_stream_service_controller"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.set_test_stream_service_controller">[docs]</a> <span class="k">def</span> <span class="nf">set_test_stream_service_controller</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">controller</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Sets the test stream service controller that has started a gRPC server</span>
<span class="sd"> serving the test stream for any job started from the given user defined</span>
<span class="sd"> pipeline.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_test_stream_service_controllers</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">pipeline</span><span class="p">))]</span> <span class="o">=</span> <span class="n">controller</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.get_test_stream_service_controller"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.get_test_stream_service_controller">[docs]</a> <span class="k">def</span> <span class="nf">get_test_stream_service_controller</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Gets the test stream service controller that has started a gRPC server</span>
<span class="sd"> serving the test stream for any job started from the given user defined</span>
<span class="sd"> 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">_test_stream_service_controllers</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)),</span> <span class="kc">None</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.evict_test_stream_service_controller"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.evict_test_stream_service_controller">[docs]</a> <span class="k">def</span> <span class="nf">evict_test_stream_service_controller</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Evicts and pops the test stream service controller that has started a</span>
<span class="sd"> gRPC server serving the test stream for any job started from the given</span>
<span class="sd"> user defined pipeline. Noop if the given pipeline is absent from the</span>
<span class="sd"> environment. If no pipeline is specified, evicts for all pipelines.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_test_stream_service_controllers</span><span class="o">.</span><span class="n">pop</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">pipeline</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">_test_stream_service_controllers</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.is_terminated"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.is_terminated">[docs]</a> <span class="k">def</span> <span class="nf">is_terminated</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Queries if the most recent job (by executing the given pipeline) state</span>
<span class="sd"> is in a terminal state. True if absent.&quot;&quot;&quot;</span>
<span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline_result</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="k">if</span> <span class="n">result</span><span class="p">:</span>
<span class="k">return</span> <span class="n">runner</span><span class="o">.</span><span class="n">PipelineState</span><span class="o">.</span><span class="n">is_terminal</span><span class="p">(</span><span class="n">result</span><span class="o">.</span><span class="n">state</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">True</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.set_cached_source_signature"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.set_cached_source_signature">[docs]</a> <span class="k">def</span> <span class="nf">set_cached_source_signature</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">signature</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_cached_source_signature</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">pipeline</span><span class="p">))]</span> <span class="o">=</span> <span class="n">signature</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.get_cached_source_signature"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.get_cached_source_signature">[docs]</a> <span class="k">def</span> <span class="nf">get_cached_source_signature</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cached_source_signature</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)),</span> <span class="nb">set</span><span class="p">())</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.evict_cached_source_signature"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.evict_cached_source_signature">[docs]</a> <span class="k">def</span> <span class="nf">evict_cached_source_signature</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Evicts the signature generated for each recorded source of the given</span>
<span class="sd"> pipeline. Noop if the given pipeline is absent from the environment. If no</span>
<span class="sd"> pipeline is specified, evicts for all pipelines.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cached_source_signature</span><span class="o">.</span><span class="n">pop</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">pipeline</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">_cached_source_signature</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.track_user_pipelines"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.track_user_pipelines">[docs]</a> <span class="k">def</span> <span class="nf">track_user_pipelines</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Record references to all user defined pipeline instances watched in</span>
<span class="sd"> current environment.</span>
<span class="sd"> Current static global singleton interactive environment holds references to</span>
<span class="sd"> a set of pipeline instances defined by the user in the watched scope.</span>
<span class="sd"> Interactive Beam features could use the references to determine if a given</span>
<span class="sd"> pipeline is defined by user or implicitly created by Beam SDK or runners,</span>
<span class="sd"> then handle them differently.</span>
<span class="sd"> This is invoked every time a PTransform is to be applied if the current</span>
<span class="sd"> code execution is under ipython due to the possibility that any user defined</span>
<span class="sd"> pipeline can be re-evaluated through notebook cell re-execution at any time.</span>
<span class="sd"> Each time this is invoked, it will check if there is a cache manager</span>
<span class="sd"> already created for each user defined pipeline. If not, create one for it.</span>
<span class="sd"> If a pipeline is no longer watched due to re-execution while its</span>
<span class="sd"> PCollections are still in watched scope, the pipeline becomes anonymous but</span>
<span class="sd"> still accessible indirectly through references to its PCollections. This</span>
<span class="sd"> function also clears up internal states for those anonymous pipelines once</span>
<span class="sd"> all their PCollections are anonymous.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">for</span> <span class="n">watching</span> <span class="ow">in</span> <span class="bp">self</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">pipeline</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">_tracked_user_pipelines</span><span class="o">.</span><span class="n">add_user_pipeline</span><span class="p">(</span><span class="n">val</span><span class="p">)</span>
<span class="n">_</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span><span class="n">val</span><span class="p">,</span> <span class="n">create_if_absent</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">_</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">get_recording_manager</span><span class="p">(</span><span class="n">val</span><span class="p">,</span> <span class="n">create_if_absent</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">all_tracked_pipeline_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">_background_caching_jobs</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span><span class="o">.</span><span class="n">union</span><span class="p">(</span>
<span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_test_stream_service_controllers</span><span class="o">.</span><span class="n">keys</span><span class="p">()),</span>
<span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_managers</span><span class="o">.</span><span class="n">keys</span><span class="p">()),</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">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="p">))</span>
<span class="k">for</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_computed_pcolls</span><span class="p">},</span>
<span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cached_source_signature</span><span class="o">.</span><span class="n">keys</span><span class="p">()),</span>
<span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_main_pipeline_results</span><span class="o">.</span><span class="n">keys</span><span class="p">()))</span>
<span class="n">inspectable_pipelines</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_inspector</span><span class="o">.</span><span class="n">inspectable_pipelines</span>
<span class="k">for</span> <span class="n">pipeline</span> <span class="ow">in</span> <span class="n">all_tracked_pipeline_ids</span><span class="p">:</span>
<span class="k">if</span> <span class="n">pipeline</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">inspectable_pipelines</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">cleanup</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span></div>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">tracked_user_pipelines</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns the user pipelines in this environment.&quot;&quot;&quot;</span>
<span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_tracked_user_pipelines</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">p</span>
<div class="viewcode-block" id="InteractiveEnvironment.user_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.user_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">user_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">derived_pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Returns the user pipeline for the given derived pipeline.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_tracked_user_pipelines</span><span class="o">.</span><span class="n">get_user_pipeline</span><span class="p">(</span><span class="n">derived_pipeline</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.add_user_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.add_user_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">add_user_pipeline</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="bp">self</span><span class="o">.</span><span class="n">_tracked_user_pipelines</span><span class="o">.</span><span class="n">add_user_pipeline</span><span class="p">(</span><span class="n">user_pipeline</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.add_derived_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.add_derived_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">add_derived_pipeline</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">derived_pipeline</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Adds the derived pipeline to the parent user pipeline.&quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_tracked_user_pipelines</span><span class="o">.</span><span class="n">add_derived_pipeline</span><span class="p">(</span>
<span class="n">user_pipeline</span><span class="p">,</span> <span class="n">derived_pipeline</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.evict_tracked_pipelines"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.evict_tracked_pipelines">[docs]</a> <span class="k">def</span> <span class="nf">evict_tracked_pipelines</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="w"> </span><span class="sd">&quot;&quot;&quot;Evicts the user pipeline and its derived pipelines.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">user_pipeline</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_tracked_user_pipelines</span><span class="o">.</span><span class="n">evict</span><span class="p">(</span><span class="n">user_pipeline</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_tracked_user_pipelines</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.pipeline_id_to_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.pipeline_id_to_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">pipeline_id_to_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pid</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Converts a pipeline id to a user 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">_tracked_user_pipelines</span><span class="o">.</span><span class="n">get_pipeline</span><span class="p">(</span><span class="n">pid</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.mark_pcollection_computed"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.mark_pcollection_computed">[docs]</a> <span class="k">def</span> <span class="nf">mark_pcollection_computed</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="w"> </span><span class="sd">&quot;&quot;&quot;Marks computation completeness for the given pcolls.</span>
<span class="sd"> Interactive Beam can use this information to determine if a computation is</span>
<span class="sd"> needed to introspect the data of any given PCollection.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_computed_pcolls</span><span class="o">.</span><span class="n">update</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="p">)</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.evict_computed_pcollections"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.evict_computed_pcollections">[docs]</a> <span class="k">def</span> <span class="nf">evict_computed_pcollections</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Evicts all computed PCollections for the given pipeline. If no pipeline</span>
<span class="sd"> is specified, evicts for all pipelines.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="n">discarded</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="k">for</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_computed_pcolls</span><span class="p">:</span>
<span class="k">if</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span> <span class="ow">is</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="n">discarded</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="bp">self</span><span class="o">.</span><span class="n">_computed_pcolls</span> <span class="o">-=</span> <span class="n">discarded</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_computed_pcolls</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span></div>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">computed_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">_computed_pcolls</span>
<div class="viewcode-block" id="InteractiveEnvironment.load_jquery_with_datatable"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.load_jquery_with_datatable">[docs]</a> <span class="k">def</span> <span class="nf">load_jquery_with_datatable</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Loads common resources to enable jquery with datatable configured for</span>
<span class="sd"> notebook frontends if necessary. If the resources have been loaded, NOOP.</span>
<span class="sd"> A window.interactive_beam_jquery with datatable plugin configured can be</span>
<span class="sd"> used in following notebook cells once this is invoked.</span>
<span class="sd"> #. There should only be one jQuery imported.</span>
<span class="sd"> #. Datatable needs to be imported after jQuery is loaded.</span>
<span class="sd"> #. Imported jQuery is attached to window named as jquery[version].</span>
<span class="sd"> #. The window attachment needs to happen at the end of import chain until</span>
<span class="sd"> all jQuery plugins are set.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">try</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">IPython.display</span> <span class="kn">import</span> <span class="n">Javascript</span>
<span class="kn">from</span> <span class="nn">IPython.display</span> <span class="kn">import</span> <span class="n">display_javascript</span>
<span class="n">display_javascript</span><span class="p">(</span>
<span class="n">Javascript</span><span class="p">(</span>
<span class="n">_JQUERY_WITH_DATATABLE_TEMPLATE</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">customized_script</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">)))</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="k">pass</span> <span class="c1"># NOOP if dependencies are not available.</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.import_html_to_head"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.import_html_to_head">[docs]</a> <span class="k">def</span> <span class="nf">import_html_to_head</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">html_hrefs</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Imports given external HTMLs (supported through webcomponents) into</span>
<span class="sd"> the head of the document.</span>
<span class="sd"> On load of webcomponentsjs, import given HTMLs. If HTML import is already</span>
<span class="sd"> supported, skip loading webcomponentsjs.</span>
<span class="sd"> No matter how many times an HTML import occurs in the document, only the</span>
<span class="sd"> first occurrence really embeds the external HTML. In a notebook environment,</span>
<span class="sd"> the body of the document is always changing due to cell [re-]execution,</span>
<span class="sd"> deletion and re-ordering. Thus, HTML imports shouldn&#39;t be put in the body</span>
<span class="sd"> especially the output areas of notebook cells.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">try</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">IPython.display</span> <span class="kn">import</span> <span class="n">Javascript</span>
<span class="kn">from</span> <span class="nn">IPython.display</span> <span class="kn">import</span> <span class="n">display_javascript</span>
<span class="n">display_javascript</span><span class="p">(</span>
<span class="n">Javascript</span><span class="p">(</span><span class="n">_HTML_IMPORT_TEMPLATE</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">hrefs</span><span class="o">=</span><span class="n">html_hrefs</span><span class="p">)))</span>
<span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span>
<span class="k">pass</span> <span class="c1"># NOOP if dependencies are not available.</span></div>
<div class="viewcode-block" id="InteractiveEnvironment.get_sql_chain"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_environment.html#apache_beam.runners.interactive.interactive_environment.InteractiveEnvironment.get_sql_chain">[docs]</a> <span class="k">def</span> <span class="nf">get_sql_chain</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">set_user_pipeline</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="k">if</span> <span class="n">pipeline</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">sql_chain</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">sql_chain</span><span class="p">[</span><span class="n">pipeline</span><span class="p">]</span> <span class="o">=</span> <span class="n">SqlChain</span><span class="p">()</span>
<span class="n">chain</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">sql_chain</span><span class="p">[</span><span class="n">pipeline</span><span class="p">]</span>
<span class="k">if</span> <span class="n">set_user_pipeline</span><span class="p">:</span>
<span class="k">if</span> <span class="n">chain</span><span class="o">.</span><span class="n">user_pipeline</span> <span class="ow">and</span> <span class="n">chain</span><span class="o">.</span><span class="n">user_pipeline</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">pipeline</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;The beam_sql magic tries to query PCollections from multiple &#39;</span>
<span class="s1">&#39;pipelines: </span><span class="si">%s</span><span class="s1"> and </span><span class="si">%s</span><span class="s1">&#39;</span><span class="p">,</span>
<span class="n">chain</span><span class="o">.</span><span class="n">user_pipeline</span><span class="p">,</span>
<span class="n">pipeline</span><span class="p">)</span>
<span class="n">chain</span><span class="o">.</span><span class="n">user_pipeline</span> <span class="o">=</span> <span class="n">pipeline</span>
<span class="k">return</span> <span class="n">chain</span></div>
<span class="k">def</span> <span class="nf">_get_gcs_cache_dir</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">cache_dir</span><span class="p">):</span>
<span class="n">cache_dir_path</span> <span class="o">=</span> <span class="n">PurePath</span><span class="p">(</span><span class="n">cache_dir</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">cache_dir_path</span><span class="o">.</span><span class="n">parts</span><span class="p">)</span> <span class="o">&lt;</span> <span class="mi">2</span><span class="p">:</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">error</span><span class="p">(</span>
<span class="s1">&#39;GCS bucket cache path &quot;</span><span class="si">%s</span><span class="s1">&quot; is too short to be valid. See &#39;</span>
<span class="s1">&#39;https://cloud.google.com/storage/docs/naming-buckets for &#39;</span>
<span class="s1">&#39;the expected format.&#39;</span><span class="p">,</span>
<span class="n">cache_dir</span><span class="p">)</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;cache_root GCS bucket path is invalid.&#39;</span><span class="p">)</span>
<span class="n">bucket_name</span> <span class="o">=</span> <span class="n">cache_dir_path</span><span class="o">.</span><span class="n">parts</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span>
<span class="n">assert_bucket_exists</span><span class="p">(</span><span class="n">bucket_name</span><span class="p">)</span>
<span class="k">return</span> <span class="s1">&#39;gs://</span><span class="si">{}</span><span class="s1">/</span><span class="si">{}</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="s1">&#39;/&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">cache_dir_path</span><span class="o">.</span><span class="n">parts</span><span class="p">[</span><span class="mi">1</span><span class="p">:]),</span> <span class="nb">id</span><span class="p">(</span><span class="n">pipeline</span><span class="p">))</span></div>
</pre></div>
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/rtfd/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.Navigation.enable(true);
});
</script>
</body>
</html>