blob: 9fcba4481731b1862efd1a34407bbe1cae1203a1 [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_runner &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_runner</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_runner</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;A runner that allows running of Beam pipelines interactively.</span>
<span class="sd">This module is experimental. No backwards-compatibility guarantees.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Optional</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</span> <span class="kn">import</span> <span class="n">runners</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="kn">import</span> <span class="n">FlinkRunnerOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="kn">import</span> <span class="n">GoogleCloudOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="kn">import</span> <span class="n">PipelineOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="kn">import</span> <span class="n">WorkerOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="kn">import</span> <span class="n">PipelineVisitor</span>
<span class="kn">from</span> <span class="nn">apache_beam.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">interactive_environment</span> <span class="k">as</span> <span class="n">ie</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="kn">import</span> <span class="n">pipeline_instrument</span> <span class="k">as</span> <span class="n">inst</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive</span> <span class="kn">import</span> <span class="n">background_caching_job</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.dataproc.types</span> <span class="kn">import</span> <span class="n">ClusterMetadata</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.display</span> <span class="kn">import</span> <span class="n">pipeline_graph</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.options</span> <span class="kn">import</span> <span class="n">capture_control</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.utils</span> <span class="kn">import</span> <span class="n">to_element_list</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.utils</span> <span class="kn">import</span> <span class="n">watch_sources</span>
<span class="kn">from</span> <span class="nn">apache_beam.testing.test_stream_service</span> <span class="kn">import</span> <span class="n">TestStreamServiceController</span>
<span class="c1"># size of PCollection samples cached.</span>
<span class="n">SAMPLE_SIZE</span> <span class="o">=</span> <span class="mi">8</span>
<span class="n">_LOGGER</span> <span class="o">=</span> <span class="n">logging</span><span class="o">.</span><span class="n">getLogger</span><span class="p">(</span><span class="vm">__name__</span><span class="p">)</span>
<div class="viewcode-block" id="InteractiveRunner"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner">[docs]</a><span class="k">class</span> <span class="nc">InteractiveRunner</span><span class="p">(</span><span class="n">runners</span><span class="o">.</span><span class="n">PipelineRunner</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;An interactive runner for Beam Python pipelines.</span>
<span class="sd"> Allows interactively building and running Beam Python pipelines.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">underlying_runner</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">render_option</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">skip_display</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="n">force_compute</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="n">blocking</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Constructor of InteractiveRunner.</span>
<span class="sd"> Args:</span>
<span class="sd"> underlying_runner: (runner.PipelineRunner)</span>
<span class="sd"> render_option: (str) this parameter decides how the pipeline graph is</span>
<span class="sd"> rendered. See display.pipeline_graph_renderer for available options.</span>
<span class="sd"> skip_display: (bool) whether to skip display operations when running the</span>
<span class="sd"> pipeline. Useful if running large pipelines when display is not</span>
<span class="sd"> needed.</span>
<span class="sd"> force_compute: (bool) whether sequential pipeline runs can use cached data</span>
<span class="sd"> of PCollections computed from the previous runs including show API</span>
<span class="sd"> invocation from interactive_beam module. If True, always run the whole</span>
<span class="sd"> pipeline and compute data for PCollections forcefully. If False, use</span>
<span class="sd"> available data and run minimum pipeline fragment to only compute data</span>
<span class="sd"> not available.</span>
<span class="sd"> blocking: (bool) whether the pipeline run should be blocking or not.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">underlying_runner</span> <span class="ow">or</span> <span class="n">direct_runner</span><span class="o">.</span><span class="n">DirectRunner</span><span class="p">())</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_render_option</span> <span class="o">=</span> <span class="n">render_option</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_skip_display</span> <span class="o">=</span> <span class="n">skip_display</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_force_compute</span> <span class="o">=</span> <span class="n">force_compute</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_blocking</span> <span class="o">=</span> <span class="n">blocking</span>
<div class="viewcode-block" id="InteractiveRunner.is_fnapi_compatible"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.is_fnapi_compatible">[docs]</a> <span class="k">def</span> <span class="nf">is_fnapi_compatible</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># TODO(https://github.com/apache/beam/issues/19937):</span>
<span class="c1"># return self._underlying_runner.is_fnapi_compatible()</span>
<span class="k">return</span> <span class="kc">False</span></div>
<div class="viewcode-block" id="InteractiveRunner.set_render_option"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.set_render_option">[docs]</a> <span class="k">def</span> <span class="nf">set_render_option</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">render_option</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Sets the rendering option.</span>
<span class="sd"> Args:</span>
<span class="sd"> render_option: (str) this parameter decides how the pipeline graph is</span>
<span class="sd"> rendered. See display.pipeline_graph_renderer for available options.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_render_option</span> <span class="o">=</span> <span class="n">render_option</span></div>
<div class="viewcode-block" id="InteractiveRunner.start_session"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.start_session">[docs]</a> <span class="k">def</span> <span class="nf">start_session</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Start the session that keeps back-end managers and workers alive.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span><span class="p">:</span>
<span class="k">return</span>
<span class="n">enter</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span> <span class="s1">&#39;__enter__&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">if</span> <span class="n">enter</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</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;Starting session.&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">enter</span><span class="p">()</span>
<span class="k">else</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;Keep alive not supported.&#39;</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveRunner.end_session"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.end_session">[docs]</a> <span class="k">def</span> <span class="nf">end_session</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;End the session that keeps backend managers and workers alive.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span><span class="p">:</span>
<span class="k">return</span>
<span class="n">exit</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span> <span class="s1">&#39;__exit__&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">if</span> <span class="n">exit</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_in_session</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">_LOGGER</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">&#39;Ending session.&#39;</span><span class="p">)</span>
<span class="n">exit</span><span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveRunner.apply"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.apply">[docs]</a> <span class="k">def</span> <span class="nf">apply</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">,</span> <span class="n">pvalueish</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># TODO(qinyeli, BEAM-646): Remove runner interception of apply.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">pvalueish</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span></div>
<div class="viewcode-block" id="InteractiveRunner.run_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.run_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">run_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="n">options</span><span class="p">):</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">enable_recording_replay</span><span class="p">:</span>
<span class="n">capture_control</span><span class="o">.</span><span class="n">evict_captured_data</span><span class="p">()</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_force_compute</span><span class="p">:</span>
<span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">evict_computed_pcollections</span><span class="p">()</span>
<span class="c1"># Make sure that sources without a user reference are still cached.</span>
<span class="n">watch_sources</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="n">user_pipeline</span> <span class="o">=</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">user_pipeline</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.portability.flink_runner</span> <span class="kn">import</span> <span class="n">FlinkRunner</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span> <span class="n">FlinkRunner</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">configure_for_flink</span><span class="p">(</span><span class="n">user_pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span>
<span class="n">pipeline_instrument</span> <span class="o">=</span> <span class="n">inst</span><span class="o">.</span><span class="n">build_pipeline_instrument</span><span class="p">(</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span>
<span class="c1"># The user_pipeline analyzed might be None if the pipeline given has nothing</span>
<span class="c1"># to be cached and tracing back to the user defined pipeline is impossible.</span>
<span class="c1"># When it&#39;s None, there is no need to cache including the background</span>
<span class="c1"># caching job and no result to track since no background caching job is</span>
<span class="c1"># started at all.</span>
<span class="k">if</span> <span class="n">user_pipeline</span><span class="p">:</span>
<span class="c1"># Should use the underlying runner and run asynchronously.</span>
<span class="n">background_caching_job</span><span class="o">.</span><span class="n">attempt_to_run_background_caching_job</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span> <span class="n">user_pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="n">background_caching_job</span><span class="o">.</span><span class="n">has_source_to_cache</span><span class="p">(</span><span class="n">user_pipeline</span><span class="p">)</span> <span class="ow">and</span>
<span class="ow">not</span> <span class="n">background_caching_job</span><span class="o">.</span><span class="n">is_a_test_stream_service_running</span><span class="p">(</span>
<span class="n">user_pipeline</span><span class="p">)):</span>
<span class="n">streaming_cache_manager</span> <span class="o">=</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span>
<span class="n">user_pipeline</span><span class="p">)</span>
<span class="c1"># Only make the server if it doesn&#39;t exist already.</span>
<span class="k">if</span> <span class="p">(</span><span class="n">streaming_cache_manager</span> <span class="ow">and</span>
<span class="ow">not</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">get_test_stream_service_controller</span><span class="p">(</span>
<span class="n">user_pipeline</span><span class="p">)):</span>
<span class="k">def</span> <span class="nf">exception_handler</span><span class="p">(</span><span class="n">e</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="nb">str</span><span class="p">(</span><span class="n">e</span><span class="p">))</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="n">test_stream_service</span> <span class="o">=</span> <span class="n">TestStreamServiceController</span><span class="p">(</span>
<span class="n">streaming_cache_manager</span><span class="p">,</span> <span class="n">exception_handler</span><span class="o">=</span><span class="n">exception_handler</span><span class="p">)</span>
<span class="n">test_stream_service</span><span class="o">.</span><span class="n">start</span><span class="p">()</span>
<span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">set_test_stream_service_controller</span><span class="p">(</span>
<span class="n">user_pipeline</span><span class="p">,</span> <span class="n">test_stream_service</span><span class="p">)</span>
<span class="n">pipeline_to_execute</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">Pipeline</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span>
<span class="n">pipeline_instrument</span><span class="o">.</span><span class="n">instrumented_pipeline_proto</span><span class="p">(),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_runner</span><span class="p">,</span>
<span class="n">options</span><span class="p">)</span>
<span class="k">if</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">get_test_stream_service_controller</span><span class="p">(</span><span class="n">user_pipeline</span><span class="p">):</span>
<span class="n">endpoint</span> <span class="o">=</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">get_test_stream_service_controller</span><span class="p">(</span>
<span class="n">user_pipeline</span><span class="p">)</span><span class="o">.</span><span class="n">endpoint</span>
<span class="c1"># TODO: make the StreamingCacheManager and TestStreamServiceController</span>
<span class="c1"># constructed when the InteractiveEnvironment is imported.</span>
<span class="k">class</span> <span class="nc">TestStreamVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">visit_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">):</span>
<span class="kn">from</span> <span class="nn">apache_beam.testing.test_stream</span> <span class="kn">import</span> <span class="n">TestStream</span>
<span class="k">if</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="n">TestStream</span><span class="p">)</span> <span class="ow">and</span>
<span class="ow">not</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">_events</span><span class="p">):</span>
<span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">_endpoint</span> <span class="o">=</span> <span class="n">endpoint</span>
<span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="n">TestStreamVisitor</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">_skip_display</span><span class="p">:</span>
<span class="n">a_pipeline_graph</span> <span class="o">=</span> <span class="n">pipeline_graph</span><span class="o">.</span><span class="n">PipelineGraph</span><span class="p">(</span>
<span class="n">pipeline_instrument</span><span class="o">.</span><span class="n">original_pipeline_proto</span><span class="p">,</span>
<span class="n">render_option</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_render_option</span><span class="p">)</span>
<span class="n">a_pipeline_graph</span><span class="o">.</span><span class="n">display_graph</span><span class="p">()</span>
<span class="n">main_job_result</span> <span class="o">=</span> <span class="n">PipelineResult</span><span class="p">(</span>
<span class="n">pipeline_to_execute</span><span class="o">.</span><span class="n">run</span><span class="p">(),</span> <span class="n">pipeline_instrument</span><span class="p">)</span>
<span class="c1"># In addition to this pipeline result setting, redundant result setting from</span>
<span class="c1"># outer scopes are also recommended since the user_pipeline might not be</span>
<span class="c1"># available from within this scope.</span>
<span class="k">if</span> <span class="n">user_pipeline</span><span class="p">:</span>
<span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">set_pipeline_result</span><span class="p">(</span><span class="n">user_pipeline</span><span class="p">,</span> <span class="n">main_job_result</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_blocking</span><span class="p">:</span>
<span class="n">main_job_result</span><span class="o">.</span><span class="n">wait_until_finish</span><span class="p">()</span>
<span class="k">if</span> <span class="n">main_job_result</span><span class="o">.</span><span class="n">state</span> <span class="ow">is</span> <span class="n">beam</span><span class="o">.</span><span class="n">runners</span><span class="o">.</span><span class="n">runner</span><span class="o">.</span><span class="n">PipelineState</span><span class="o">.</span><span class="n">DONE</span><span class="p">:</span>
<span class="c1"># pylint: disable=bad-option-value</span>
<span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">mark_pcollection_computed</span><span class="p">(</span>
<span class="n">pipeline_instrument</span><span class="o">.</span><span class="n">cached_pcolls</span><span class="p">)</span>
<span class="k">return</span> <span class="n">main_job_result</span></div>
<div class="viewcode-block" id="InteractiveRunner.configure_for_flink"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.InteractiveRunner.configure_for_flink">[docs]</a> <span class="k">def</span> <span class="nf">configure_for_flink</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">user_pipeline</span><span class="p">:</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">:</span> <span class="n">PipelineOptions</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Configures the pipeline options for running a job with Flink.</span>
<span class="sd"> When running with a FlinkRunner, a job server started from an uber jar</span>
<span class="sd"> (locally built or remotely downloaded) hosting the beam_job_api will</span>
<span class="sd"> communicate with the Flink cluster located at the given flink_master in the</span>
<span class="sd"> pipeline options.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">clusters</span> <span class="o">=</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">clusters</span>
<span class="k">if</span> <span class="n">clusters</span><span class="o">.</span><span class="n">pipelines</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">user_pipeline</span><span class="p">,</span> <span class="kc">None</span><span class="p">):</span>
<span class="c1"># Noop for a known pipeline using a known Dataproc cluster.</span>
<span class="k">return</span>
<span class="n">flink_master</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_strip_protocol_if_any</span><span class="p">(</span>
<span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">FlinkRunnerOptions</span><span class="p">)</span><span class="o">.</span><span class="n">flink_master</span><span class="p">)</span>
<span class="n">cluster_metadata</span> <span class="o">=</span> <span class="n">clusters</span><span class="o">.</span><span class="n">default_cluster_metadata</span>
<span class="k">if</span> <span class="n">flink_master</span> <span class="o">==</span> <span class="s1">&#39;[auto]&#39;</span><span class="p">:</span>
<span class="c1"># Try to create/reuse a cluster when no flink_master is given.</span>
<span class="n">project_id</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">GoogleCloudOptions</span><span class="p">)</span><span class="o">.</span><span class="n">project</span>
<span class="n">region</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">GoogleCloudOptions</span><span class="p">)</span><span class="o">.</span><span class="n">region</span> <span class="ow">or</span> <span class="s1">&#39;us-central1&#39;</span>
<span class="k">if</span> <span class="n">project_id</span><span class="p">:</span>
<span class="k">if</span> <span class="n">clusters</span><span class="o">.</span><span class="n">default_cluster_metadata</span><span class="p">:</span>
<span class="c1"># Reuse the cluster name from default in case of a known cluster.</span>
<span class="n">cluster_metadata</span> <span class="o">=</span> <span class="n">ClusterMetadata</span><span class="p">(</span>
<span class="n">project_id</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span>
<span class="n">region</span><span class="o">=</span><span class="n">region</span><span class="p">,</span>
<span class="n">cluster_name</span><span class="o">=</span><span class="n">clusters</span><span class="o">.</span><span class="n">default_cluster_metadata</span><span class="o">.</span><span class="n">cluster_name</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Generate the metadata with a new unique cluster name.</span>
<span class="n">cluster_metadata</span> <span class="o">=</span> <span class="n">ClusterMetadata</span><span class="p">(</span>
<span class="n">project_id</span><span class="o">=</span><span class="n">project_id</span><span class="p">,</span> <span class="n">region</span><span class="o">=</span><span class="n">region</span><span class="p">)</span>
<span class="c1"># Add additional configurations.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_worker_options_to_cluster_metadata</span><span class="p">(</span><span class="n">options</span><span class="p">,</span> <span class="n">cluster_metadata</span><span class="p">)</span>
<span class="c1"># else use the default cluster metadata.</span>
<span class="k">elif</span> <span class="n">flink_master</span> <span class="ow">in</span> <span class="n">clusters</span><span class="o">.</span><span class="n">master_urls</span><span class="p">:</span>
<span class="n">cluster_metadata</span> <span class="o">=</span> <span class="n">clusters</span><span class="o">.</span><span class="n">cluster_metadata</span><span class="p">(</span><span class="n">flink_master</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span> <span class="c1"># Noop if a self-hosted Flink is in use.</span>
<span class="k">return</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">cluster_metadata</span><span class="p">:</span>
<span class="k">return</span> <span class="c1"># Not even a default cluster to create/reuse, run Flink locally.</span>
<span class="n">dcm</span> <span class="o">=</span> <span class="n">clusters</span><span class="o">.</span><span class="n">create</span><span class="p">(</span><span class="n">cluster_metadata</span><span class="p">)</span>
<span class="c1"># Side effects associated with the user_pipeline.</span>
<span class="n">clusters</span><span class="o">.</span><span class="n">pipelines</span><span class="p">[</span><span class="n">user_pipeline</span><span class="p">]</span> <span class="o">=</span> <span class="n">dcm</span>
<span class="n">dcm</span><span class="o">.</span><span class="n">pipelines</span><span class="o">.</span><span class="n">add</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">_configure_flink_options</span><span class="p">(</span>
<span class="n">options</span><span class="p">,</span>
<span class="n">clusters</span><span class="o">.</span><span class="n">DATAPROC_FLINK_VERSION</span><span class="p">,</span>
<span class="n">dcm</span><span class="o">.</span><span class="n">cluster_metadata</span><span class="o">.</span><span class="n">master_url</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_strip_protocol_if_any</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">flink_master</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]):</span>
<span class="k">if</span> <span class="n">flink_master</span><span class="p">:</span>
<span class="n">parts</span> <span class="o">=</span> <span class="n">flink_master</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;://&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">parts</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">return</span> <span class="n">parts</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span>
<span class="k">return</span> <span class="n">flink_master</span>
<span class="k">def</span> <span class="nf">_worker_options_to_cluster_metadata</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">options</span><span class="p">:</span> <span class="n">PipelineOptions</span><span class="p">,</span> <span class="n">cluster_metadata</span><span class="p">:</span> <span class="n">ClusterMetadata</span><span class="p">):</span>
<span class="n">worker_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">WorkerOptions</span><span class="p">)</span>
<span class="k">if</span> <span class="n">worker_options</span><span class="o">.</span><span class="n">subnetwork</span><span class="p">:</span>
<span class="n">cluster_metadata</span><span class="o">.</span><span class="n">subnetwork</span> <span class="o">=</span> <span class="n">worker_options</span><span class="o">.</span><span class="n">subnetwork</span>
<span class="k">if</span> <span class="n">worker_options</span><span class="o">.</span><span class="n">num_workers</span><span class="p">:</span>
<span class="n">cluster_metadata</span><span class="o">.</span><span class="n">num_workers</span> <span class="o">=</span> <span class="n">worker_options</span><span class="o">.</span><span class="n">num_workers</span>
<span class="k">if</span> <span class="n">worker_options</span><span class="o">.</span><span class="n">machine_type</span><span class="p">:</span>
<span class="n">cluster_metadata</span><span class="o">.</span><span class="n">machine_type</span> <span class="o">=</span> <span class="n">worker_options</span><span class="o">.</span><span class="n">machine_type</span>
<span class="k">def</span> <span class="nf">_configure_flink_options</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">options</span><span class="p">:</span> <span class="n">PipelineOptions</span><span class="p">,</span> <span class="n">flink_version</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> <span class="n">master_url</span><span class="p">:</span> <span class="nb">str</span><span class="p">):</span>
<span class="n">flink_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">FlinkRunnerOptions</span><span class="p">)</span>
<span class="n">flink_options</span><span class="o">.</span><span class="n">flink_version</span> <span class="o">=</span> <span class="n">flink_version</span>
<span class="c1"># flink_options.flink_job_server_jar will be populated by the</span>
<span class="c1"># apache_beam.utils.subprocess_server.JavaJarServer.path_to_beam_jar,</span>
<span class="c1"># do not populate it explicitly.</span>
<span class="n">flink_options</span><span class="o">.</span><span class="n">flink_master</span> <span class="o">=</span> <span class="n">master_url</span></div>
<div class="viewcode-block" id="PipelineResult"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.PipelineResult">[docs]</a><span class="k">class</span> <span class="nc">PipelineResult</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">runners</span><span class="o">.</span><span class="n">runner</span><span class="o">.</span><span class="n">PipelineResult</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Provides access to information about a pipeline.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">underlying_result</span><span class="p">,</span> <span class="n">pipeline_instrument</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Constructor of PipelineResult.</span>
<span class="sd"> Args:</span>
<span class="sd"> underlying_result: (PipelineResult) the result returned by the underlying</span>
<span class="sd"> runner running the pipeline.</span>
<span class="sd"> pipeline_instrument: (PipelineInstrument) pipeline instrument describing</span>
<span class="sd"> the pipeline being executed with interactivity applied and related</span>
<span class="sd"> metadata including where the interactivity-backing cache lies.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="n">underlying_result</span><span class="o">.</span><span class="n">state</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_result</span> <span class="o">=</span> <span class="n">underlying_result</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_instrument</span> <span class="o">=</span> <span class="n">pipeline_instrument</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">state</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">_underlying_result</span><span class="o">.</span><span class="n">state</span>
<div class="viewcode-block" id="PipelineResult.wait_until_finish"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.PipelineResult.wait_until_finish">[docs]</a> <span class="k">def</span> <span class="nf">wait_until_finish</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_result</span><span class="o">.</span><span class="n">wait_until_finish</span><span class="p">()</span></div>
<div class="viewcode-block" id="PipelineResult.get"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.PipelineResult.get">[docs]</a> <span class="k">def</span> <span class="nf">get</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">,</span> <span class="n">include_window_info</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Materializes the PCollection into a list.</span>
<span class="sd"> If include_window_info is True, then returns the elements as</span>
<span class="sd"> WindowedValues. Otherwise, return the element as itself.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">include_window_info</span><span class="p">))</span></div>
<div class="viewcode-block" id="PipelineResult.read"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.PipelineResult.read">[docs]</a> <span class="k">def</span> <span class="nf">read</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">,</span> <span class="n">include_window_info</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Reads the PCollection one element at a time from cache.</span>
<span class="sd"> If include_window_info is True, then returns the elements as</span>
<span class="sd"> WindowedValues. Otherwise, return the element as itself.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">key</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_instrument</span><span class="o">.</span><span class="n">cache_key</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span>
<span class="n">cache_manager</span> <span class="o">=</span> <span class="n">ie</span><span class="o">.</span><span class="n">current_env</span><span class="p">()</span><span class="o">.</span><span class="n">get_cache_manager</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_instrument</span><span class="o">.</span><span class="n">user_pipeline</span><span class="p">)</span>
<span class="k">if</span> <span class="n">key</span> <span class="ow">and</span> <span class="n">cache_manager</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="s1">&#39;full&#39;</span><span class="p">,</span> <span class="n">key</span><span class="p">):</span>
<span class="n">coder</span> <span class="o">=</span> <span class="n">cache_manager</span><span class="o">.</span><span class="n">load_pcoder</span><span class="p">(</span><span class="s1">&#39;full&#39;</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span>
<span class="n">reader</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">cache_manager</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="s1">&#39;full&#39;</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span>
<span class="k">return</span> <span class="n">to_element_list</span><span class="p">(</span><span class="n">reader</span><span class="p">,</span> <span class="n">coder</span><span class="p">,</span> <span class="n">include_window_info</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;PCollection not available, please run the pipeline.&#39;</span><span class="p">)</span></div>
<div class="viewcode-block" id="PipelineResult.cancel"><a class="viewcode-back" href="../../../../apache_beam.runners.interactive.interactive_runner.html#apache_beam.runners.interactive.interactive_runner.PipelineResult.cancel">[docs]</a> <span class="k">def</span> <span class="nf">cancel</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_underlying_result</span><span class="o">.</span><span class="n">cancel</span><span class="p">()</span></div></div>
</pre></div>
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/rtfd/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.Navigation.enable(true);
});
</script>
</body>
</html>