blob: b44856ffbd43aa6173b09f1d68e7df247c5cd977 [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.transforms.environments &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.transforms.environments</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.transforms.environments</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;Environments concepts.</span>
<span class="sd">For internal use only. No backwards compatibility guarantees.&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">import</span> <span class="nn">json</span>
<span class="kn">import</span> <span class="nn">logging</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">import</span> <span class="nn">tempfile</span>
<span class="kn">from</span> <span class="nn">types</span> <span class="kn">import</span> <span class="n">MappingProxyType</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TYPE_CHECKING</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Callable</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Dict</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Iterable</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Iterator</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">List</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Mapping</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Optional</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Set</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Tuple</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Type</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TypeVar</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Union</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">overload</span>
<span class="kn">from</span> <span class="nn">google.protobuf</span> <span class="kn">import</span> <span class="n">message</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">coders</span>
<span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="kn">import</span> <span class="n">SetupOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="kn">import</span> <span class="n">common_urns</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="kn">import</span> <span class="n">python_urns</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">beam_runner_api_pb2</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">endpoints_pb2</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.portability</span> <span class="kn">import</span> <span class="n">stager</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.portability.sdk_container_builder</span> <span class="kn">import</span> <span class="n">SdkContainerImageBuilder</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.resources</span> <span class="kn">import</span> <span class="n">resource_hints_from_options</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="kn">import</span> <span class="n">proto_utils</span>
<span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</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">PortableOptions</span>
<span class="kn">from</span> <span class="nn">apache_beam.runners.pipeline_context</span> <span class="kn">import</span> <span class="n">PipelineContext</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span>
<span class="s1">&#39;Environment&#39;</span><span class="p">,</span>
<span class="s1">&#39;DefaultEnvironment&#39;</span><span class="p">,</span>
<span class="s1">&#39;DockerEnvironment&#39;</span><span class="p">,</span>
<span class="s1">&#39;ProcessEnvironment&#39;</span><span class="p">,</span>
<span class="s1">&#39;ExternalEnvironment&#39;</span><span class="p">,</span>
<span class="s1">&#39;EmbeddedPythonEnvironment&#39;</span><span class="p">,</span>
<span class="s1">&#39;EmbeddedPythonGrpcEnvironment&#39;</span><span class="p">,</span>
<span class="s1">&#39;SubprocessSDKEnvironment&#39;</span><span class="p">,</span>
<span class="s1">&#39;PyPIArtifactRegistry&#39;</span>
<span class="p">]</span>
<span class="n">T</span> <span class="o">=</span> <span class="n">TypeVar</span><span class="p">(</span><span class="s1">&#39;T&#39;</span><span class="p">)</span>
<span class="n">EnvironmentT</span> <span class="o">=</span> <span class="n">TypeVar</span><span class="p">(</span><span class="s1">&#39;EnvironmentT&#39;</span><span class="p">,</span> <span class="n">bound</span><span class="o">=</span><span class="s1">&#39;Environment&#39;</span><span class="p">)</span>
<span class="n">ConstructorFn</span> <span class="o">=</span> <span class="n">Callable</span><span class="p">[[</span>
<span class="n">Optional</span><span class="p">[</span><span class="n">Any</span><span class="p">],</span>
<span class="n">Iterable</span><span class="p">[</span><span class="nb">str</span><span class="p">],</span>
<span class="n">Iterable</span><span class="p">[</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ArtifactInformation</span><span class="p">],</span>
<span class="n">Mapping</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">],</span>
<span class="s1">&#39;PipelineContext&#39;</span>
<span class="p">],</span>
<span class="n">Any</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">looks_like_json</span><span class="p">(</span><span class="n">s</span><span class="p">):</span>
<span class="kn">import</span> <span class="nn">re</span>
<span class="k">return</span> <span class="n">re</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="sa">r</span><span class="s1">&#39;\s*\{.*\}\s*$&#39;</span><span class="p">,</span> <span class="n">s</span><span class="p">)</span>
<span class="n">APACHE_BEAM_DOCKER_IMAGE_PREFIX</span> <span class="o">=</span> <span class="s1">&#39;apache/beam&#39;</span>
<span class="n">APACHE_BEAM_JAVA_CONTAINER_NAME_PREFIX</span> <span class="o">=</span> <span class="s1">&#39;beam_java&#39;</span>
<span class="n">SDK_VERSION_CAPABILITY_PREFIX</span> <span class="o">=</span> <span class="s1">&#39;beam:version:sdk_base:&#39;</span>
<span class="k">def</span> <span class="nf">is_apache_beam_container</span><span class="p">(</span><span class="n">container_image</span><span class="p">):</span>
<span class="k">return</span> <span class="n">container_image</span> <span class="ow">and</span> <span class="n">container_image</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span>
<span class="n">APACHE_BEAM_DOCKER_IMAGE_PREFIX</span><span class="p">)</span>
<div class="viewcode-block" id="Environment"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment">[docs]</a><span class="k">class</span> <span class="nc">Environment</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Abstract base class for environments.</span>
<span class="sd"> Represents a type and configuration of environment.</span>
<span class="sd"> Each type of Environment should have a unique urn.</span>
<span class="sd"> For internal use only. No backwards compatibility guarantees.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">_known_urns</span> <span class="o">=</span> <span class="p">{}</span> <span class="c1"># type: Dict[str, Tuple[Optional[type], ConstructorFn]]</span>
<span class="n">_urn_to_env_cls</span> <span class="o">=</span> <span class="p">{}</span> <span class="c1"># type: Dict[str, type]</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">capabilities</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: Optional[Mapping[str, bytes]]</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_capabilities</span> <span class="o">=</span> <span class="n">capabilities</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_artifacts</span> <span class="o">=</span> <span class="nb">sorted</span><span class="p">(</span><span class="n">artifacts</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">())</span>
<span class="c1"># Hints on created environments should be immutable since pipeline context</span>
<span class="c1"># stores environments in hash maps and we use hints to compute the hash.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_resource_hints</span> <span class="o">=</span> <span class="n">MappingProxyType</span><span class="p">(</span>
<span class="nb">dict</span><span class="p">(</span><span class="n">resource_hints</span><span class="p">)</span> <span class="k">if</span> <span class="n">resource_hints</span> <span class="k">else</span> <span class="p">{})</span>
<span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="vm">__class__</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="vm">__class__</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_artifacts</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">_artifacts</span>
<span class="c1"># Assuming that we don&#39;t have instances of the same Environment subclass</span>
<span class="c1"># with different set of capabilities.</span>
<span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_resource_hints</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">_resource_hints</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; int</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">((</span><span class="bp">self</span><span class="o">.</span><span class="vm">__class__</span><span class="p">,</span> <span class="nb">frozenset</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_resource_hints</span><span class="o">.</span><span class="n">items</span><span class="p">())))</span>
<div class="viewcode-block" id="Environment.artifacts"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.artifacts">[docs]</a> <span class="k">def</span> <span class="nf">artifacts</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_artifacts</span></div>
<div class="viewcode-block" id="Environment.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; Tuple[str, Optional[Union[message.Message, bytes, str]]]</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="Environment.capabilities"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.capabilities">[docs]</a> <span class="k">def</span> <span class="nf">capabilities</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; Iterable[str]</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_capabilities</span></div>
<div class="viewcode-block" id="Environment.resource_hints"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.resource_hints">[docs]</a> <span class="k">def</span> <span class="nf">resource_hints</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; Mapping[str, bytes]</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_resource_hints</span></div>
<span class="nd">@classmethod</span>
<span class="nd">@overload</span>
<span class="k">def</span> <span class="nf">register_urn</span><span class="p">(</span>
<span class="bp">cls</span><span class="p">,</span>
<span class="n">urn</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">parameter_type</span><span class="p">,</span> <span class="c1"># type: Type[T]</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; Callable[[Union[type, Callable[[T, Iterable[str], PipelineContext], Any]]], Callable[[T, Iterable[str], PipelineContext], Any]]</span>
<span class="k">pass</span>
<span class="nd">@classmethod</span>
<span class="nd">@overload</span>
<span class="k">def</span> <span class="nf">register_urn</span><span class="p">(</span>
<span class="bp">cls</span><span class="p">,</span>
<span class="n">urn</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">parameter_type</span><span class="p">,</span> <span class="c1"># type: None</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; Callable[[Union[type, Callable[[bytes, Iterable[str], Iterable[beam_runner_api_pb2.ArtifactInformation], PipelineContext], Any]]], Callable[[bytes, Iterable[str], PipelineContext], Any]]</span>
<span class="k">pass</span>
<span class="nd">@classmethod</span>
<span class="nd">@overload</span>
<span class="k">def</span> <span class="nf">register_urn</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span>
<span class="n">urn</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">parameter_type</span><span class="p">,</span> <span class="c1"># type: Type[T]</span>
<span class="n">constructor</span> <span class="c1"># type: Callable[[T, Iterable[str], Iterable[beam_runner_api_pb2.ArtifactInformation], PipelineContext], Any]</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; None</span>
<span class="k">pass</span>
<span class="nd">@classmethod</span>
<span class="nd">@overload</span>
<span class="k">def</span> <span class="nf">register_urn</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span>
<span class="n">urn</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">parameter_type</span><span class="p">,</span> <span class="c1"># type: None</span>
<span class="n">constructor</span> <span class="c1"># type: Callable[[bytes, Iterable[str], Iterable[beam_runner_api_pb2.ArtifactInformation], PipelineContext], Any]</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; None</span>
<span class="k">pass</span>
<div class="viewcode-block" id="Environment.register_urn"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.register_urn">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">register_urn</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">urn</span><span class="p">,</span> <span class="n">parameter_type</span><span class="p">,</span> <span class="n">constructor</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">register</span><span class="p">(</span><span class="n">constructor</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">constructor</span><span class="p">,</span> <span class="nb">type</span><span class="p">):</span>
<span class="n">constructor</span><span class="o">.</span><span class="n">from_runner_api_parameter</span> <span class="o">=</span> <span class="n">register</span><span class="p">(</span>
<span class="n">constructor</span><span class="o">.</span><span class="n">from_runner_api_parameter</span><span class="p">)</span>
<span class="c1"># register environment urn to environment class</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_urn_to_env_cls</span><span class="p">[</span><span class="n">urn</span><span class="p">]</span> <span class="o">=</span> <span class="n">constructor</span>
<span class="k">return</span> <span class="n">constructor</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_known_urns</span><span class="p">[</span><span class="n">urn</span><span class="p">]</span> <span class="o">=</span> <span class="n">parameter_type</span><span class="p">,</span> <span class="n">constructor</span>
<span class="k">return</span> <span class="nb">staticmethod</span><span class="p">(</span><span class="n">constructor</span><span class="p">)</span>
<span class="k">if</span> <span class="n">constructor</span><span class="p">:</span>
<span class="c1"># Used as a statement.</span>
<span class="n">register</span><span class="p">(</span><span class="n">constructor</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Used as a decorator.</span>
<span class="k">return</span> <span class="n">register</span></div>
<div class="viewcode-block" id="Environment.get_env_cls_from_urn"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.get_env_cls_from_urn">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">get_env_cls_from_urn</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">urn</span><span class="p">):</span>
<span class="c1"># type: (str) -&gt; Type[Environment]</span>
<span class="k">return</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_urn_to_env_cls</span><span class="p">[</span><span class="n">urn</span><span class="p">]</span></div>
<div class="viewcode-block" id="Environment.to_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.to_runner_api">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; beam_runner_api_pb2.Environment</span>
<span class="n">urn</span><span class="p">,</span> <span class="n">typed_param</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_runner_api_parameter</span><span class="p">(</span><span class="n">context</span><span class="p">)</span>
<span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">Environment</span><span class="p">(</span>
<span class="n">urn</span><span class="o">=</span><span class="n">urn</span><span class="p">,</span>
<span class="n">payload</span><span class="o">=</span><span class="n">typed_param</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">()</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">typed_param</span><span class="p">,</span> <span class="n">message</span><span class="o">.</span><span class="n">Message</span><span class="p">)</span> <span class="k">else</span> <span class="n">typed_param</span> <span class="k">if</span>
<span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">typed_param</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">)</span> <span class="ow">or</span>
<span class="n">typed_param</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">)</span> <span class="k">else</span> <span class="n">typed_param</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="s1">&#39;utf-8&#39;</span><span class="p">),</span>
<span class="n">capabilities</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">capabilities</span><span class="p">(),</span>
<span class="n">dependencies</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">artifacts</span><span class="p">(),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">resource_hints</span><span class="p">())</span></div>
<div class="viewcode-block" id="Environment.from_runner_api"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.from_runner_api">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span>
<span class="n">proto</span><span class="p">,</span> <span class="c1"># type: Optional[beam_runner_api_pb2.Environment]</span>
<span class="n">context</span> <span class="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; Optional[Environment]</span>
<span class="k">if</span> <span class="n">proto</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="ow">not</span> <span class="n">proto</span><span class="o">.</span><span class="n">urn</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="n">parameter_type</span><span class="p">,</span> <span class="n">constructor</span> <span class="o">=</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_known_urns</span><span class="p">[</span><span class="n">proto</span><span class="o">.</span><span class="n">urn</span><span class="p">]</span>
<span class="k">return</span> <span class="n">constructor</span><span class="p">(</span>
<span class="n">proto_utils</span><span class="o">.</span><span class="n">parse_Bytes</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">payload</span><span class="p">,</span> <span class="n">parameter_type</span><span class="p">),</span>
<span class="n">proto</span><span class="o">.</span><span class="n">capabilities</span><span class="p">,</span>
<span class="n">proto</span><span class="o">.</span><span class="n">dependencies</span><span class="p">,</span>
<span class="n">proto</span><span class="o">.</span><span class="n">resource_hints</span><span class="p">,</span>
<span class="n">context</span><span class="p">)</span></div>
<div class="viewcode-block" id="Environment.from_options"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.Environment.from_options">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_options</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># type: (Type[EnvironmentT], PortableOptions) -&gt; EnvironmentT</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Creates an Environment object from PortableOptions.</span>
<span class="sd"> Args:</span>
<span class="sd"> options: The PortableOptions object.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div></div>
<div class="viewcode-block" id="DefaultEnvironment"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DefaultEnvironment">[docs]</a><span class="nd">@Environment</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">common_urns</span><span class="o">.</span><span class="n">environments</span><span class="o">.</span><span class="n">DEFAULT</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">DefaultEnvironment</span><span class="p">(</span><span class="n">Environment</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Used as a stub when context is missing a default environment.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="DefaultEnvironment.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DefaultEnvironment.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="k">return</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">environments</span><span class="o">.</span><span class="n">DEFAULT</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span></div>
<div class="viewcode-block" id="DefaultEnvironment.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DefaultEnvironment.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">payload</span><span class="p">,</span> <span class="c1"># type: beam_runner_api_pb2.DockerPayload</span>
<span class="n">capabilities</span><span class="p">,</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="p">,</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="p">,</span> <span class="c1"># type: Mapping[str, bytes]</span>
<span class="n">context</span> <span class="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; DefaultEnvironment</span>
<span class="k">return</span> <span class="n">DefaultEnvironment</span><span class="p">(</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">capabilities</span><span class="p">,</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">artifacts</span><span class="p">,</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="DockerEnvironment"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DockerEnvironment">[docs]</a><span class="nd">@Environment</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">environments</span><span class="o">.</span><span class="n">DOCKER</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">DockerPayload</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">DockerEnvironment</span><span class="p">(</span><span class="n">Environment</span><span class="p">):</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">container_image</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: Optional[str]</span>
<span class="n">capabilities</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: Optional[Mapping[str, bytes]]</span>
<span class="p">):</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">capabilities</span><span class="p">,</span> <span class="n">artifacts</span><span class="p">,</span> <span class="n">resource_hints</span><span class="p">)</span>
<span class="k">if</span> <span class="n">container_image</span><span class="p">:</span>
<span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span>
<span class="s1">&#39;Using provided Python SDK container image: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">container_image</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">container_image</span> <span class="o">=</span> <span class="n">container_image</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">&#39;No image given, using default Python SDK image&#39;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">container_image</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">default_docker_image</span><span class="p">()</span>
<span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span>
<span class="s1">&#39;Python SDK container image set to &quot;</span><span class="si">%s</span><span class="s1">&quot; for Docker environment&#39;</span> <span class="o">%</span>
<span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">container_image</span><span class="p">))</span>
<span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__eq__</span><span class="p">(</span><span class="n">other</span><span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">container_image</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">container_image</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">((</span><span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__hash__</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">container_image</span><span class="p">))</span>
<span class="k">def</span> <span class="fm">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;DockerEnvironment(container_image=</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">container_image</span>
<div class="viewcode-block" id="DockerEnvironment.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DockerEnvironment.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; Tuple[str, beam_runner_api_pb2.DockerPayload]</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">environments</span><span class="o">.</span><span class="n">DOCKER</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">DockerPayload</span><span class="p">(</span><span class="n">container_image</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">container_image</span><span class="p">))</span></div>
<div class="viewcode-block" id="DockerEnvironment.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DockerEnvironment.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">payload</span><span class="p">,</span> <span class="c1"># type: beam_runner_api_pb2.DockerPayload</span>
<span class="n">capabilities</span><span class="p">,</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="p">,</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="p">,</span> <span class="c1"># type: Mapping[str, bytes]</span>
<span class="n">context</span> <span class="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; DockerEnvironment</span>
<span class="k">return</span> <span class="n">DockerEnvironment</span><span class="p">(</span>
<span class="n">container_image</span><span class="o">=</span><span class="n">payload</span><span class="o">.</span><span class="n">container_image</span><span class="p">,</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">capabilities</span><span class="p">,</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">artifacts</span><span class="p">,</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints</span><span class="p">)</span></div>
<div class="viewcode-block" id="DockerEnvironment.from_options"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DockerEnvironment.from_options">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_options</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># type: (PortableOptions) -&gt; DockerEnvironment</span>
<span class="k">if</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">SetupOptions</span><span class="p">)</span><span class="o">.</span><span class="n">prebuild_sdk_container_engine</span><span class="p">:</span>
<span class="n">prebuilt_container_image</span> <span class="o">=</span> <span class="n">SdkContainerImageBuilder</span><span class="o">.</span><span class="n">build_container_image</span><span class="p">(</span>
<span class="n">options</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">cls</span><span class="o">.</span><span class="n">from_container_image</span><span class="p">(</span>
<span class="n">container_image</span><span class="o">=</span><span class="n">prebuilt_container_image</span><span class="p">,</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints_from_options</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="p">)</span>
<span class="k">return</span> <span class="bp">cls</span><span class="o">.</span><span class="n">from_container_image</span><span class="p">(</span>
<span class="n">container_image</span><span class="o">=</span><span class="n">options</span><span class="o">.</span><span class="n">lookup_environment_option</span><span class="p">(</span>
<span class="s1">&#39;docker_container_image&#39;</span><span class="p">)</span> <span class="ow">or</span> <span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">,</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints_from_options</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DockerEnvironment.from_container_image"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DockerEnvironment.from_container_image">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_container_image</span><span class="p">(</span>
<span class="bp">cls</span><span class="p">,</span> <span class="n">container_image</span><span class="p">,</span> <span class="n">artifacts</span><span class="o">=</span><span class="p">(),</span> <span class="n">resource_hints</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="c1"># type: (str, Iterable[beam_runner_api_pb2.ArtifactInformation], Optional[Mapping[str, bytes]]) -&gt; DockerEnvironment</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">container_image</span><span class="o">=</span><span class="n">container_image</span><span class="p">,</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_docker_capabilities</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">artifacts</span><span class="p">,</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints</span><span class="p">)</span></div>
<div class="viewcode-block" id="DockerEnvironment.default_docker_image"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.DockerEnvironment.default_docker_image">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">default_docker_image</span><span class="p">():</span>
<span class="c1"># type: () -&gt; str</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">version</span> <span class="k">as</span> <span class="n">beam_version</span>
<span class="n">sdk_version</span> <span class="o">=</span> <span class="n">beam_version</span><span class="o">.</span><span class="n">__version__</span>
<span class="n">version_suffix</span> <span class="o">=</span> <span class="s1">&#39;.&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">([</span><span class="nb">str</span><span class="p">(</span><span class="n">i</span><span class="p">)</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">sys</span><span class="o">.</span><span class="n">version_info</span><span class="p">[</span><span class="mi">0</span><span class="p">:</span><span class="mi">2</span><span class="p">]])</span>
<span class="n">image</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">APACHE_BEAM_DOCKER_IMAGE_PREFIX</span> <span class="o">+</span>
<span class="s1">&#39;_python</span><span class="si">{version_suffix}</span><span class="s1">_sdk:</span><span class="si">{tag}</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">version_suffix</span><span class="o">=</span><span class="n">version_suffix</span><span class="p">,</span> <span class="n">tag</span><span class="o">=</span><span class="n">sdk_version</span><span class="p">))</span>
<span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">&#39;Default Python SDK image for environment is </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">image</span><span class="p">))</span>
<span class="k">return</span> <span class="n">image</span></div></div>
<div class="viewcode-block" id="ProcessEnvironment"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ProcessEnvironment">[docs]</a><span class="nd">@Environment</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">environments</span><span class="o">.</span><span class="n">PROCESS</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ProcessPayload</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">ProcessEnvironment</span><span class="p">(</span><span class="n">Environment</span><span class="p">):</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">command</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">os</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">arch</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">env</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: Optional[Mapping[str, str]]</span>
<span class="n">capabilities</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: Optional[Mapping[str, bytes]]</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; None</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">capabilities</span><span class="p">,</span> <span class="n">artifacts</span><span class="p">,</span> <span class="n">resource_hints</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">command</span> <span class="o">=</span> <span class="n">command</span>
<span class="bp">self</span><span class="o">.</span><span class="n">os</span> <span class="o">=</span> <span class="n">os</span>
<span class="bp">self</span><span class="o">.</span><span class="n">arch</span> <span class="o">=</span> <span class="n">arch</span>
<span class="bp">self</span><span class="o">.</span><span class="n">env</span> <span class="o">=</span> <span class="n">env</span> <span class="ow">or</span> <span class="p">{}</span>
<span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__eq__</span><span class="p">(</span><span class="n">other</span><span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">command</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">command</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">os</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">os</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">arch</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">arch</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">env</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">env</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; int</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">((</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__hash__</span><span class="p">(),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">command</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">os</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">arch</span><span class="p">,</span>
<span class="nb">frozenset</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">env</span><span class="o">.</span><span class="n">items</span><span class="p">())))</span>
<span class="k">def</span> <span class="fm">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; str</span>
<span class="n">repr_parts</span> <span class="o">=</span> <span class="p">[</span><span class="s1">&#39;command=</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">command</span><span class="p">]</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">os</span><span class="p">:</span>
<span class="n">repr_parts</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="s1">&#39;os=</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">os</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">arch</span><span class="p">:</span>
<span class="n">repr_parts</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="s1">&#39;arch=</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">arch</span><span class="p">)</span>
<span class="n">repr_parts</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="s1">&#39;env=</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">env</span><span class="p">)</span>
<span class="k">return</span> <span class="s1">&#39;ProcessEnvironment(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="s1">&#39;,&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">repr_parts</span><span class="p">)</span>
<div class="viewcode-block" id="ProcessEnvironment.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ProcessEnvironment.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; Tuple[str, beam_runner_api_pb2.ProcessPayload]</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">environments</span><span class="o">.</span><span class="n">PROCESS</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ProcessPayload</span><span class="p">(</span>
<span class="n">os</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">os</span><span class="p">,</span> <span class="n">arch</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">arch</span><span class="p">,</span> <span class="n">command</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">command</span><span class="p">,</span> <span class="n">env</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">env</span><span class="p">))</span></div>
<div class="viewcode-block" id="ProcessEnvironment.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ProcessEnvironment.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">payload</span><span class="p">,</span>
<span class="n">capabilities</span><span class="p">,</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="p">,</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="p">,</span> <span class="c1"># type: Mapping[str, bytes]</span>
<span class="n">context</span> <span class="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; ProcessEnvironment</span>
<span class="k">return</span> <span class="n">ProcessEnvironment</span><span class="p">(</span>
<span class="n">command</span><span class="o">=</span><span class="n">payload</span><span class="o">.</span><span class="n">command</span><span class="p">,</span>
<span class="n">os</span><span class="o">=</span><span class="n">payload</span><span class="o">.</span><span class="n">os</span><span class="p">,</span>
<span class="n">arch</span><span class="o">=</span><span class="n">payload</span><span class="o">.</span><span class="n">arch</span><span class="p">,</span>
<span class="n">env</span><span class="o">=</span><span class="n">payload</span><span class="o">.</span><span class="n">env</span><span class="p">,</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">capabilities</span><span class="p">,</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">artifacts</span><span class="p">,</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints</span><span class="p">,</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="ProcessEnvironment.parse_environment_variables"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ProcessEnvironment.parse_environment_variables">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">parse_environment_variables</span><span class="p">(</span><span class="n">variables</span><span class="p">):</span>
<span class="n">env</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">var</span> <span class="ow">in</span> <span class="n">variables</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">name</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">var</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="mi">1</span><span class="p">)</span>
<span class="n">env</span><span class="p">[</span><span class="n">name</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span>
<span class="k">except</span> <span class="ne">ValueError</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Invalid process_variables &quot;</span><span class="si">%s</span><span class="s1">&quot; (expected assignment in the &#39;</span>
<span class="s1">&#39;form &quot;FOO=bar&quot;).&#39;</span> <span class="o">%</span> <span class="n">var</span><span class="p">)</span>
<span class="k">return</span> <span class="n">env</span></div>
<div class="viewcode-block" id="ProcessEnvironment.from_options"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ProcessEnvironment.from_options">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_options</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># type: (PortableOptions) -&gt; ProcessEnvironment</span>
<span class="k">if</span> <span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">:</span>
<span class="n">config</span> <span class="o">=</span> <span class="n">json</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span><span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;command&#39;</span><span class="p">),</span>
<span class="n">os</span><span class="o">=</span><span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;os&#39;</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">),</span>
<span class="n">arch</span><span class="o">=</span><span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;arch&#39;</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">),</span>
<span class="n">env</span><span class="o">=</span><span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;env&#39;</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">),</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints_from_options</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="p">)</span>
<span class="n">env</span> <span class="o">=</span> <span class="bp">cls</span><span class="o">.</span><span class="n">parse_environment_variables</span><span class="p">(</span>
<span class="n">options</span><span class="o">.</span><span class="n">lookup_environment_option</span><span class="p">(</span><span class="s1">&#39;process_variables&#39;</span><span class="p">)</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="n">options</span><span class="o">.</span><span class="n">lookup_environment_option</span><span class="p">(</span><span class="s1">&#39;process_variables&#39;</span><span class="p">)</span> <span class="k">else</span> <span class="p">[])</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">options</span><span class="o">.</span><span class="n">lookup_environment_option</span><span class="p">(</span><span class="s1">&#39;process_command&#39;</span><span class="p">),</span>
<span class="n">env</span><span class="o">=</span><span class="n">env</span><span class="p">,</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints_from_options</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="p">)</span></div></div>
<div class="viewcode-block" id="ExternalEnvironment"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ExternalEnvironment">[docs]</a><span class="nd">@Environment</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">environments</span><span class="o">.</span><span class="n">EXTERNAL</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ExternalPayload</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">ExternalEnvironment</span><span class="p">(</span><span class="n">Environment</span><span class="p">):</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">url</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">params</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: Optional[Mapping[str, str]]</span>
<span class="n">capabilities</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: Optional[Mapping[str, bytes]]</span>
<span class="p">):</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">capabilities</span><span class="p">,</span> <span class="n">artifacts</span><span class="p">,</span> <span class="n">resource_hints</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">url</span> <span class="o">=</span> <span class="n">url</span>
<span class="bp">self</span><span class="o">.</span><span class="n">params</span> <span class="o">=</span> <span class="n">params</span>
<span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__eq__</span><span class="p">(</span><span class="n">other</span><span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">url</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">url</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">params</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">params</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; int</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">((</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__hash__</span><span class="p">(),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">url</span><span class="p">,</span>
<span class="nb">frozenset</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">params</span><span class="o">.</span><span class="n">items</span><span class="p">())</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">params</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">else</span> <span class="kc">None</span><span class="p">))</span>
<span class="k">def</span> <span class="fm">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; str</span>
<span class="k">return</span> <span class="s1">&#39;ExternalEnvironment(url=</span><span class="si">%s</span><span class="s1">,params=</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">url</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">params</span><span class="p">)</span>
<div class="viewcode-block" id="ExternalEnvironment.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ExternalEnvironment.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; Tuple[str, beam_runner_api_pb2.ExternalPayload]</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">common_urns</span><span class="o">.</span><span class="n">environments</span><span class="o">.</span><span class="n">EXTERNAL</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span>
<span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">ExternalPayload</span><span class="p">(</span>
<span class="n">endpoint</span><span class="o">=</span><span class="n">endpoints_pb2</span><span class="o">.</span><span class="n">ApiServiceDescriptor</span><span class="p">(</span><span class="n">url</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">url</span><span class="p">),</span>
<span class="n">params</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">params</span><span class="p">))</span></div>
<div class="viewcode-block" id="ExternalEnvironment.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ExternalEnvironment.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">payload</span><span class="p">,</span> <span class="c1"># type: beam_runner_api_pb2.ExternalPayload</span>
<span class="n">capabilities</span><span class="p">,</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="p">,</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="p">,</span> <span class="c1"># type: Mapping[str, bytes]</span>
<span class="n">context</span> <span class="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; ExternalEnvironment</span>
<span class="k">return</span> <span class="n">ExternalEnvironment</span><span class="p">(</span>
<span class="n">payload</span><span class="o">.</span><span class="n">endpoint</span><span class="o">.</span><span class="n">url</span><span class="p">,</span>
<span class="n">params</span><span class="o">=</span><span class="n">payload</span><span class="o">.</span><span class="n">params</span> <span class="ow">or</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">capabilities</span><span class="p">,</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">artifacts</span><span class="p">,</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints</span><span class="p">)</span></div>
<div class="viewcode-block" id="ExternalEnvironment.from_options"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.ExternalEnvironment.from_options">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_options</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># type: (PortableOptions) -&gt; ExternalEnvironment</span>
<span class="k">if</span> <span class="n">looks_like_json</span><span class="p">(</span><span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">):</span>
<span class="n">config</span> <span class="o">=</span> <span class="n">json</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span><span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">)</span>
<span class="n">url</span> <span class="o">=</span> <span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;url&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">url</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;External environment endpoint must be set.&#39;</span><span class="p">)</span>
<span class="n">params</span> <span class="o">=</span> <span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;params&#39;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">:</span>
<span class="n">url</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">environment_config</span>
<span class="n">params</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">url</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">lookup_environment_option</span><span class="p">(</span><span class="s1">&#39;external_service_address&#39;</span><span class="p">)</span>
<span class="n">params</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">url</span><span class="p">,</span>
<span class="n">params</span><span class="o">=</span><span class="n">params</span><span class="p">,</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints_from_options</span><span class="p">(</span><span class="n">options</span><span class="p">))</span></div></div>
<div class="viewcode-block" id="EmbeddedPythonEnvironment"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonEnvironment">[docs]</a><span class="nd">@Environment</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">python_urns</span><span class="o">.</span><span class="n">EMBEDDED_PYTHON</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">EmbeddedPythonEnvironment</span><span class="p">(</span><span class="n">Environment</span><span class="p">):</span>
<div class="viewcode-block" id="EmbeddedPythonEnvironment.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonEnvironment.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; Tuple[str, None]</span>
<span class="k">return</span> <span class="n">python_urns</span><span class="o">.</span><span class="n">EMBEDDED_PYTHON</span><span class="p">,</span> <span class="kc">None</span></div>
<div class="viewcode-block" id="EmbeddedPythonEnvironment.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonEnvironment.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_payload</span><span class="p">,</span> <span class="c1"># type: None</span>
<span class="n">capabilities</span><span class="p">,</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="p">,</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="p">,</span> <span class="c1"># type: Mapping[str, bytes]</span>
<span class="n">context</span> <span class="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; EmbeddedPythonEnvironment</span>
<span class="k">return</span> <span class="n">EmbeddedPythonEnvironment</span><span class="p">(</span><span class="n">capabilities</span><span class="p">,</span> <span class="n">artifacts</span><span class="p">,</span> <span class="n">resource_hints</span><span class="p">)</span></div>
<div class="viewcode-block" id="EmbeddedPythonEnvironment.from_options"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonEnvironment.from_options">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_options</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># type: (PortableOptions) -&gt; EmbeddedPythonEnvironment</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints_from_options</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="EmbeddedPythonEnvironment.default"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonEnvironment.default">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">default</span><span class="p">(</span><span class="bp">cls</span><span class="p">):</span>
<span class="c1"># type: () -&gt; EmbeddedPythonEnvironment</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span><span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span> <span class="n">artifacts</span><span class="o">=</span><span class="p">())</span></div></div>
<div class="viewcode-block" id="EmbeddedPythonGrpcEnvironment"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonGrpcEnvironment">[docs]</a><span class="nd">@Environment</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">python_urns</span><span class="o">.</span><span class="n">EMBEDDED_PYTHON_GRPC</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">EmbeddedPythonGrpcEnvironment</span><span class="p">(</span><span class="n">Environment</span><span class="p">):</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">state_cache_size</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">data_buffer_time_limit_ms</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">capabilities</span><span class="o">=</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="p">(),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="p">):</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">capabilities</span><span class="p">,</span> <span class="n">artifacts</span><span class="p">,</span> <span class="n">resource_hints</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state_cache_size</span> <span class="o">=</span> <span class="n">state_cache_size</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data_buffer_time_limit_ms</span> <span class="o">=</span> <span class="n">data_buffer_time_limit_ms</span>
<span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__eq__</span><span class="p">(</span><span class="n">other</span><span class="p">)</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state_cache_size</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">state_cache_size</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data_buffer_time_limit_ms</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">data_buffer_time_limit_ms</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; int</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">((</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__hash__</span><span class="p">(),</span>
<span class="bp">self</span><span class="o">.</span><span class="n">state_cache_size</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data_buffer_time_limit_ms</span><span class="p">))</span>
<span class="k">def</span> <span class="fm">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; str</span>
<span class="n">repr_parts</span> <span class="o">=</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">state_cache_size</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">repr_parts</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="s1">&#39;state_cache_size=</span><span class="si">%d</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">state_cache_size</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">data_buffer_time_limit_ms</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">repr_parts</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="s1">&#39;data_buffer_time_limit_ms=</span><span class="si">%d</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">data_buffer_time_limit_ms</span><span class="p">)</span>
<span class="k">return</span> <span class="s1">&#39;EmbeddedPythonGrpcEnvironment(</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="s1">&#39;,&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">repr_parts</span><span class="p">)</span>
<div class="viewcode-block" id="EmbeddedPythonGrpcEnvironment.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonGrpcEnvironment.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; Tuple[str, bytes]</span>
<span class="n">params</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">state_cache_size</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">params</span><span class="p">[</span><span class="s1">&#39;state_cache_size&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">state_cache_size</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">data_buffer_time_limit_ms</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">params</span><span class="p">[</span><span class="s1">&#39;data_buffer_time_limit_ms&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">data_buffer_time_limit_ms</span>
<span class="n">payload</span> <span class="o">=</span> <span class="n">json</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">params</span><span class="p">)</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="s1">&#39;utf-8&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">python_urns</span><span class="o">.</span><span class="n">EMBEDDED_PYTHON_GRPC</span><span class="p">,</span> <span class="n">payload</span></div>
<div class="viewcode-block" id="EmbeddedPythonGrpcEnvironment.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonGrpcEnvironment.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">payload</span><span class="p">,</span> <span class="c1"># type: bytes</span>
<span class="n">capabilities</span><span class="p">,</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="p">,</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="p">,</span> <span class="c1"># type: Mapping[str, bytes]</span>
<span class="n">context</span> <span class="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; EmbeddedPythonGrpcEnvironment</span>
<span class="k">if</span> <span class="n">payload</span><span class="p">:</span>
<span class="n">config</span> <span class="o">=</span> <span class="n">EmbeddedPythonGrpcEnvironment</span><span class="o">.</span><span class="n">parse_config</span><span class="p">(</span>
<span class="n">payload</span><span class="o">.</span><span class="n">decode</span><span class="p">(</span><span class="s1">&#39;utf-8&#39;</span><span class="p">))</span>
<span class="k">return</span> <span class="n">EmbeddedPythonGrpcEnvironment</span><span class="p">(</span>
<span class="n">state_cache_size</span><span class="o">=</span><span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;state_cache_size&#39;</span><span class="p">),</span>
<span class="n">data_buffer_time_limit_ms</span><span class="o">=</span><span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;data_buffer_time_limit_ms&#39;</span><span class="p">),</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">capabilities</span><span class="p">,</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">artifacts</span><span class="p">,</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">EmbeddedPythonGrpcEnvironment</span><span class="p">()</span></div>
<div class="viewcode-block" id="EmbeddedPythonGrpcEnvironment.from_options"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonGrpcEnvironment.from_options">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_options</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># type: (PortableOptions) -&gt; EmbeddedPythonGrpcEnvironment</span>
<span class="k">if</span> <span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">:</span>
<span class="n">config</span> <span class="o">=</span> <span class="n">EmbeddedPythonGrpcEnvironment</span><span class="o">.</span><span class="n">parse_config</span><span class="p">(</span>
<span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">state_cache_size</span><span class="o">=</span><span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;state_cache_size&#39;</span><span class="p">),</span>
<span class="n">data_buffer_time_limit_ms</span><span class="o">=</span><span class="n">config</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">&#39;data_buffer_time_limit_ms&#39;</span><span class="p">),</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints_from_options</span><span class="p">(</span><span class="n">options</span><span class="p">))</span></div>
<div class="viewcode-block" id="EmbeddedPythonGrpcEnvironment.parse_config"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonGrpcEnvironment.parse_config">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">parse_config</span><span class="p">(</span><span class="n">s</span><span class="p">):</span>
<span class="c1"># type: (str) -&gt; Dict[str, Any]</span>
<span class="k">if</span> <span class="n">looks_like_json</span><span class="p">(</span><span class="n">s</span><span class="p">):</span>
<span class="n">config_dict</span> <span class="o">=</span> <span class="n">json</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span><span class="n">s</span><span class="p">)</span>
<span class="k">if</span> <span class="s1">&#39;state_cache_size&#39;</span> <span class="ow">in</span> <span class="n">config_dict</span><span class="p">:</span>
<span class="n">config_dict</span><span class="p">[</span><span class="s1">&#39;state_cache_size&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="nb">int</span><span class="p">(</span><span class="n">config_dict</span><span class="p">[</span><span class="s1">&#39;state_cache_size&#39;</span><span class="p">])</span>
<span class="k">if</span> <span class="s1">&#39;data_buffer_time_limit_ms&#39;</span> <span class="ow">in</span> <span class="n">config_dict</span><span class="p">:</span>
<span class="n">config_dict</span><span class="p">[</span><span class="s1">&#39;data_buffer_time_limit_ms&#39;</span><span class="p">]</span> <span class="o">=</span> \
<span class="nb">int</span><span class="p">(</span><span class="n">config_dict</span><span class="p">[</span><span class="s1">&#39;data_buffer_time_limit_ms&#39;</span><span class="p">])</span>
<span class="k">return</span> <span class="n">config_dict</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="p">{</span><span class="s1">&#39;state_cache_size&#39;</span><span class="p">:</span> <span class="nb">int</span><span class="p">(</span><span class="n">s</span><span class="p">)}</span></div>
<div class="viewcode-block" id="EmbeddedPythonGrpcEnvironment.default"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.EmbeddedPythonGrpcEnvironment.default">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">default</span><span class="p">(</span><span class="bp">cls</span><span class="p">):</span>
<span class="c1"># type: () -&gt; EmbeddedPythonGrpcEnvironment</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span><span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span> <span class="n">artifacts</span><span class="o">=</span><span class="p">())</span></div></div>
<div class="viewcode-block" id="SubprocessSDKEnvironment"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.SubprocessSDKEnvironment">[docs]</a><span class="nd">@Environment</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">python_urns</span><span class="o">.</span><span class="n">SUBPROCESS_SDK</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">SubprocessSDKEnvironment</span><span class="p">(</span><span class="n">Environment</span><span class="p">):</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">command_string</span><span class="p">,</span> <span class="c1"># type: str</span>
<span class="n">capabilities</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># type: Optional[Mapping[str, bytes]]</span>
<span class="p">):</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">capabilities</span><span class="p">,</span> <span class="n">artifacts</span><span class="p">,</span> <span class="n">resource_hints</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">command_string</span> <span class="o">=</span> <span class="n">command_string</span>
<span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__eq__</span><span class="p">(</span><span class="n">other</span><span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">command_string</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">command_string</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__hash__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; int</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">((</span><span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__hash__</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">command_string</span><span class="p">))</span>
<span class="k">def</span> <span class="fm">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; str</span>
<span class="k">return</span> <span class="s1">&#39;SubprocessSDKEnvironment(command_string=</span><span class="si">%s</span><span class="s1">)&#39;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">command_string</span>
<div class="viewcode-block" id="SubprocessSDKEnvironment.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.SubprocessSDKEnvironment.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">context</span><span class="p">):</span>
<span class="c1"># type: (PipelineContext) -&gt; Tuple[str, bytes]</span>
<span class="k">return</span> <span class="n">python_urns</span><span class="o">.</span><span class="n">SUBPROCESS_SDK</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">command_string</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="s1">&#39;utf-8&#39;</span><span class="p">)</span></div>
<div class="viewcode-block" id="SubprocessSDKEnvironment.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.SubprocessSDKEnvironment.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">payload</span><span class="p">,</span> <span class="c1"># type: bytes</span>
<span class="n">capabilities</span><span class="p">,</span> <span class="c1"># type: Iterable[str]</span>
<span class="n">artifacts</span><span class="p">,</span> <span class="c1"># type: Iterable[beam_runner_api_pb2.ArtifactInformation]</span>
<span class="n">resource_hints</span><span class="p">,</span> <span class="c1"># type: Mapping[str, bytes]</span>
<span class="n">context</span> <span class="c1"># type: PipelineContext</span>
<span class="p">):</span>
<span class="c1"># type: (...) -&gt; SubprocessSDKEnvironment</span>
<span class="k">return</span> <span class="n">SubprocessSDKEnvironment</span><span class="p">(</span>
<span class="n">payload</span><span class="o">.</span><span class="n">decode</span><span class="p">(</span><span class="s1">&#39;utf-8&#39;</span><span class="p">),</span> <span class="n">capabilities</span><span class="p">,</span> <span class="n">artifacts</span><span class="p">,</span> <span class="n">resource_hints</span><span class="p">)</span></div>
<div class="viewcode-block" id="SubprocessSDKEnvironment.from_options"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.SubprocessSDKEnvironment.from_options">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_options</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span>
<span class="c1"># type: (PortableOptions) -&gt; SubprocessSDKEnvironment</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">options</span><span class="o">.</span><span class="n">environment_config</span><span class="p">,</span>
<span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span>
<span class="n">artifacts</span><span class="o">=</span><span class="n">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">),</span>
<span class="n">resource_hints</span><span class="o">=</span><span class="n">resource_hints_from_options</span><span class="p">(</span><span class="n">options</span><span class="p">))</span></div>
<div class="viewcode-block" id="SubprocessSDKEnvironment.from_command_string"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.SubprocessSDKEnvironment.from_command_string">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">from_command_string</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">command_string</span><span class="p">):</span>
<span class="c1"># type: (str) -&gt; SubprocessSDKEnvironment</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">command_string</span><span class="p">,</span> <span class="n">capabilities</span><span class="o">=</span><span class="n">python_sdk_capabilities</span><span class="p">(),</span> <span class="n">artifacts</span><span class="o">=</span><span class="p">())</span></div></div>
<div class="viewcode-block" id="PyPIArtifactRegistry"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.PyPIArtifactRegistry">[docs]</a><span class="k">class</span> <span class="nc">PyPIArtifactRegistry</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="n">_registered_artifacts</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> <span class="c1"># type: Set[Tuple[str, str]]</span>
<div class="viewcode-block" id="PyPIArtifactRegistry.register_artifact"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.PyPIArtifactRegistry.register_artifact">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">register_artifact</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">version</span><span class="p">):</span>
<span class="bp">cls</span><span class="o">.</span><span class="n">_registered_artifacts</span><span class="o">.</span><span class="n">add</span><span class="p">((</span><span class="n">name</span><span class="p">,</span> <span class="n">version</span><span class="p">))</span></div>
<div class="viewcode-block" id="PyPIArtifactRegistry.get_artifacts"><a class="viewcode-back" href="../../../apache_beam.transforms.environments.html#apache_beam.transforms.environments.PyPIArtifactRegistry.get_artifacts">[docs]</a> <span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">get_artifacts</span><span class="p">(</span><span class="bp">cls</span><span class="p">):</span>
<span class="k">for</span> <span class="n">artifact</span> <span class="ow">in</span> <span class="bp">cls</span><span class="o">.</span><span class="n">_registered_artifacts</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">artifact</span></div></div>
<span class="k">def</span> <span class="nf">python_sdk_capabilities</span><span class="p">():</span>
<span class="c1"># type: () -&gt; List[str]</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="n">_python_sdk_capabilities_iter</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">python_sdk_docker_capabilities</span><span class="p">():</span>
<span class="k">return</span> <span class="n">python_sdk_capabilities</span><span class="p">()</span> <span class="o">+</span> <span class="p">[</span><span class="n">common_urns</span><span class="o">.</span><span class="n">protocols</span><span class="o">.</span><span class="n">SIBLING_WORKERS</span><span class="o">.</span><span class="n">urn</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">sdk_base_version_capability</span><span class="p">():</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">SDK_VERSION_CAPABILITY_PREFIX</span> <span class="o">+</span> <span class="n">DockerEnvironment</span><span class="o">.</span><span class="n">default_docker_image</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">_python_sdk_capabilities_iter</span><span class="p">():</span>
<span class="c1"># type: () -&gt; Iterator[str]</span>
<span class="k">for</span> <span class="n">urn_spec</span> <span class="ow">in</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="vm">__dict__</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="k">if</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">urn_spec</span><span class="p">,</span> <span class="s1">&#39;urn&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> <span class="ow">in</span> <span class="n">coders</span><span class="o">.</span><span class="n">Coder</span><span class="o">.</span><span class="n">_known_urns</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">urn_spec</span><span class="o">.</span><span class="n">urn</span>
<span class="k">yield</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">protocols</span><span class="o">.</span><span class="n">LEGACY_PROGRESS_REPORTING</span><span class="o">.</span><span class="n">urn</span>
<span class="k">yield</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">protocols</span><span class="o">.</span><span class="n">HARNESS_MONITORING_INFOS</span><span class="o">.</span><span class="n">urn</span>
<span class="k">yield</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">protocols</span><span class="o">.</span><span class="n">WORKER_STATUS</span><span class="o">.</span><span class="n">urn</span>
<span class="k">yield</span> <span class="n">python_urns</span><span class="o">.</span><span class="n">PACKED_COMBINE_FN</span>
<span class="k">yield</span> <span class="n">sdk_base_version_capability</span><span class="p">()</span>
<span class="k">yield</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">sdf_components</span><span class="o">.</span><span class="n">TRUNCATE_SIZED_RESTRICTION</span><span class="o">.</span><span class="n">urn</span>
<span class="k">yield</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">TO_STRING</span><span class="o">.</span><span class="n">urn</span>
<span class="k">yield</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">protocols</span><span class="o">.</span><span class="n">DATA_SAMPLING</span><span class="o">.</span><span class="n">urn</span>
<span class="k">def</span> <span class="nf">python_sdk_dependencies</span><span class="p">(</span><span class="n">options</span><span class="p">,</span> <span class="n">tmp_dir</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">if</span> <span class="n">tmp_dir</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">tmp_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">skip_prestaged_dependencies</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">SetupOptions</span><span class="p">)</span><span class="o">.</span><span class="n">prebuild_sdk_container_engine</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="k">return</span> <span class="n">stager</span><span class="o">.</span><span class="n">Stager</span><span class="o">.</span><span class="n">create_job_resources</span><span class="p">(</span>
<span class="n">options</span><span class="p">,</span>
<span class="n">tmp_dir</span><span class="p">,</span>
<span class="n">pypi_requirements</span><span class="o">=</span><span class="p">[</span>
<span class="n">artifact</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">+</span> <span class="n">artifact</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span>
<span class="k">for</span> <span class="n">artifact</span> <span class="ow">in</span> <span class="n">PyPIArtifactRegistry</span><span class="o">.</span><span class="n">get_artifacts</span><span class="p">()</span>
<span class="p">],</span>
<span class="n">skip_prestaged_dependencies</span><span class="o">=</span><span class="n">skip_prestaged_dependencies</span><span class="p">)</span>
</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>