| |
| |
| <!DOCTYPE html> |
| <!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]--> |
| <!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]--> |
| <head> |
| <meta charset="utf-8"> |
| |
| <meta name="viewport" content="width=device-width, initial-scale=1.0"> |
| |
| <title>apache_beam.runners.dataflow.dataflow_runner — Apache Beam documentation</title> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| <link rel="stylesheet" href="../../../../_static/css/theme.css" type="text/css" /> |
| |
| |
| |
| |
| |
| <link rel="index" title="Index" |
| href="../../../../genindex.html"/> |
| <link rel="search" title="Search" href="../../../../search.html"/> |
| <link rel="top" title="Apache Beam documentation" href="../../../../index.html"/> |
| <link rel="up" title="Module code" href="../../../index.html"/> |
| |
| |
| <script src="../../../../_static/js/modernizr.min.js"></script> |
| |
| </head> |
| |
| <body class="wy-body-for-nav" role="document"> |
| |
| |
| <div class="wy-grid-for-nav"> |
| |
| |
| <nav data-toggle="wy-nav-shift" class="wy-nav-side"> |
| <div class="wy-side-scroll"> |
| <div class="wy-side-nav-search"> |
| |
| |
| |
| <a href="../../../../index.html" class="icon icon-home"> Apache Beam |
| |
| |
| |
| </a> |
| |
| |
| |
| |
| |
| |
| |
| <div role="search"> |
| <form id="rtd-search-form" class="wy-form" action="../../../../search.html" method="get"> |
| <input type="text" name="q" placeholder="Search docs" /> |
| <input type="hidden" name="check_keywords" value="yes" /> |
| <input type="hidden" name="area" value="default" /> |
| </form> |
| </div> |
| |
| |
| </div> |
| |
| <div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation"> |
| |
| |
| |
| |
| |
| |
| <ul> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.coders.html">apache_beam.coders package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.internal.html">apache_beam.internal package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.io.html">apache_beam.io package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.metrics.html">apache_beam.metrics package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.options.html">apache_beam.options package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.portability.html">apache_beam.portability package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.runners.html">apache_beam.runners package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.testing.html">apache_beam.testing package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.tools.html">apache_beam.tools package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.transforms.html">apache_beam.transforms package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.typehints.html">apache_beam.typehints package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.utils.html">apache_beam.utils package</a></li> |
| </ul> |
| <ul> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.error.html">apache_beam.error module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.version.html">apache_beam.version module</a></li> |
| </ul> |
| |
| |
| |
| </div> |
| </div> |
| </nav> |
| |
| <section data-toggle="wy-nav-shift" class="wy-nav-content-wrap"> |
| |
| |
| <nav class="wy-nav-top" role="navigation" aria-label="top navigation"> |
| |
| <i data-toggle="wy-nav-top" class="fa fa-bars"></i> |
| <a href="../../../../index.html">Apache Beam</a> |
| |
| </nav> |
| |
| |
| |
| <div class="wy-nav-content"> |
| <div class="rst-content"> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| <div role="navigation" aria-label="breadcrumbs navigation"> |
| |
| <ul class="wy-breadcrumbs"> |
| |
| <li><a href="../../../../index.html">Docs</a> »</li> |
| |
| <li><a href="../../../index.html">Module code</a> »</li> |
| |
| <li>apache_beam.runners.dataflow.dataflow_runner</li> |
| |
| |
| <li class="wy-breadcrumbs-aside"> |
| |
| |
| |
| </li> |
| |
| </ul> |
| |
| |
| <hr/> |
| </div> |
| <div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article"> |
| <div itemprop="articleBody"> |
| |
| <h1>Source code for apache_beam.runners.dataflow.dataflow_runner</h1><div class="highlight"><pre> |
| <span></span><span class="c1">#</span> |
| <span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span> |
| <span class="c1"># contributor license agreements. See the NOTICE file distributed with</span> |
| <span class="c1"># this work for additional information regarding copyright ownership.</span> |
| <span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span> |
| <span class="c1"># (the "License"); you may not use this file except in compliance with</span> |
| <span class="c1"># the License. You may obtain a copy of the License at</span> |
| <span class="c1">#</span> |
| <span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span> |
| <span class="c1">#</span> |
| <span class="c1"># Unless required by applicable law or agreed to in writing, software</span> |
| <span class="c1"># distributed under the License is distributed on an "AS IS" BASIS,</span> |
| <span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span> |
| <span class="c1"># See the License for the specific language governing permissions and</span> |
| <span class="c1"># limitations under the License.</span> |
| <span class="c1">#</span> |
| |
| <span class="sd">"""A runner implementation that submits a job for remote execution.</span> |
| |
| <span class="sd">The runner will create a JSON description of the job graph and then submit it</span> |
| <span class="sd">to the Dataflow Service for remote execution by a worker.</span> |
| <span class="sd">"""</span> |
| <span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span> |
| <span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">division</span> |
| |
| <span class="kn">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">threading</span> |
| <span class="kn">import</span> <span class="nn">time</span> |
| <span class="kn">import</span> <span class="nn">traceback</span> |
| <span class="kn">import</span> <span class="nn">urllib</span> |
| <span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">hex</span> |
| <span class="kn">from</span> <span class="nn">collections</span> <span class="k">import</span> <span class="n">defaultdict</span> |
| |
| <span class="kn">from</span> <span class="nn">future.utils</span> <span class="k">import</span> <span class="n">iteritems</span> |
| |
| <span class="kn">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">coders</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">error</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">pvalue</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.internal</span> <span class="k">import</span> <span class="n">pickler</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.internal.gcp</span> <span class="k">import</span> <span class="n">json_value</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">DebugOptions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">GoogleCloudOptions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">SetupOptions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">StandardOptions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">TestOptions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="k">import</span> <span class="n">WorkerOptions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="k">import</span> <span class="n">common_urns</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pvalue</span> <span class="k">import</span> <span class="n">AsSideInput</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">names</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal.clients</span> <span class="k">import</span> <span class="n">dataflow</span> <span class="k">as</span> <span class="n">dataflow_api</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal.names</span> <span class="k">import</span> <span class="n">PropertyNames</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal.names</span> <span class="k">import</span> <span class="n">TransformNames</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.runner</span> <span class="k">import</span> <span class="n">PipelineResult</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.runner</span> <span class="k">import</span> <span class="n">PipelineRunner</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.runner</span> <span class="k">import</span> <span class="n">PipelineState</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.runner</span> <span class="k">import</span> <span class="n">PValueCache</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="k">import</span> <span class="n">window</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.display</span> <span class="k">import</span> <span class="n">DisplayData</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.typehints</span> <span class="k">import</span> <span class="n">typehints</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="k">import</span> <span class="n">proto_utils</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.utils.plugin</span> <span class="k">import</span> <span class="n">BeamPlugin</span> |
| |
| <span class="k">try</span><span class="p">:</span> <span class="c1"># Python 3</span> |
| <span class="n">unquote_to_bytes</span> <span class="o">=</span> <span class="n">urllib</span><span class="o">.</span><span class="n">parse</span><span class="o">.</span><span class="n">unquote_to_bytes</span> |
| <span class="n">quote</span> <span class="o">=</span> <span class="n">urllib</span><span class="o">.</span><span class="n">parse</span><span class="o">.</span><span class="n">quote</span> |
| <span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span> <span class="c1"># Python 2</span> |
| <span class="c1"># pylint: disable=deprecated-urllib-function</span> |
| <span class="n">unquote_to_bytes</span> <span class="o">=</span> <span class="n">urllib</span><span class="o">.</span><span class="n">unquote</span> |
| <span class="n">quote</span> <span class="o">=</span> <span class="n">urllib</span><span class="o">.</span><span class="n">quote</span> |
| |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s1">'DataflowRunner'</span><span class="p">]</span> |
| |
| |
| <div class="viewcode-block" id="DataflowRunner"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner">[docs]</a><span class="k">class</span> <span class="nc">DataflowRunner</span><span class="p">(</span><span class="n">PipelineRunner</span><span class="p">):</span> |
| <span class="sd">"""A runner that creates job graphs and submits them for remote execution.</span> |
| |
| <span class="sd"> Every execution of the run() method will submit an independent job for</span> |
| <span class="sd"> remote execution that consists of the nodes reachable from the passed in</span> |
| <span class="sd"> node argument or entire graph if node is None. The run() method returns</span> |
| <span class="sd"> after the service created the job and will not wait for the job to finish</span> |
| <span class="sd"> if blocking is set to False.</span> |
| <span class="sd"> """</span> |
| |
| <span class="c1"># A list of PTransformOverride objects to be applied before running a pipeline</span> |
| <span class="c1"># using DataflowRunner.</span> |
| <span class="c1"># Currently this only works for overrides where the input and output types do</span> |
| <span class="c1"># not change.</span> |
| <span class="c1"># For internal SDK use only. This should not be updated by Beam pipeline</span> |
| <span class="c1"># authors.</span> |
| |
| <span class="c1"># Imported here to avoid circular dependencies.</span> |
| <span class="c1"># TODO: Remove the apache_beam.pipeline dependency in CreatePTransformOverride</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.ptransform_overrides</span> <span class="k">import</span> <span class="n">CreatePTransformOverride</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.ptransform_overrides</span> <span class="k">import</span> <span class="n">ReadPTransformOverride</span> |
| |
| <span class="n">_PTRANSFORM_OVERRIDES</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">CreatePTransformOverride</span><span class="p">(),</span> |
| <span class="p">]</span> |
| |
| <span class="n">_SDF_PTRANSFORM_OVERRIDES</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">ReadPTransformOverride</span><span class="p">(),</span> |
| <span class="p">]</span> |
| |
| <span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">cache</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="c1"># Cache of CloudWorkflowStep protos generated while the runner</span> |
| <span class="c1"># "executes" a pipeline.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span> <span class="o">=</span> <span class="n">cache</span> <span class="k">if</span> <span class="n">cache</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">else</span> <span class="n">PValueCache</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_unique_step_id</span> <span class="o">=</span> <span class="mi">0</span> |
| |
| <span class="k">def</span> <span class="nf">_get_unique_step_name</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_unique_step_id</span> <span class="o">+=</span> <span class="mi">1</span> |
| <span class="k">return</span> <span class="s1">'s</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_unique_step_id</span> |
| |
| <div class="viewcode-block" id="DataflowRunner.poll_for_job_completion"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.poll_for_job_completion">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">poll_for_job_completion</span><span class="p">(</span><span class="n">runner</span><span class="p">,</span> <span class="n">result</span><span class="p">,</span> <span class="n">duration</span><span class="p">):</span> |
| <span class="sd">"""Polls for the specified job to finish running (successfully or not).</span> |
| |
| <span class="sd"> Updates the result with the new job information before returning.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> runner: DataflowRunner instance to use for polling job state.</span> |
| <span class="sd"> result: DataflowPipelineResult instance used for job information.</span> |
| <span class="sd"> duration (int): The time to wait (in milliseconds) for job to finish.</span> |
| <span class="sd"> If it is set to :data:`None`, it will wait indefinitely until the job</span> |
| <span class="sd"> is finished.</span> |
| <span class="sd"> """</span> |
| <span class="n">last_message_time</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="n">current_seen_messages</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| |
| <span class="n">last_error_rank</span> <span class="o">=</span> <span class="nb">float</span><span class="p">(</span><span class="s1">'-inf'</span><span class="p">)</span> |
| <span class="n">last_error_msg</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="n">last_job_state</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="c1"># How long to wait after pipeline failure for the error</span> |
| <span class="c1"># message to show up giving the reason for the failure.</span> |
| <span class="c1"># It typically takes about 30 seconds.</span> |
| <span class="n">final_countdown_timer_secs</span> <span class="o">=</span> <span class="mf">50.0</span> |
| <span class="n">sleep_secs</span> <span class="o">=</span> <span class="mf">5.0</span> |
| |
| <span class="c1"># Try to prioritize the user-level traceback, if any.</span> |
| <span class="k">def</span> <span class="nf">rank_error</span><span class="p">(</span><span class="n">msg</span><span class="p">):</span> |
| <span class="k">if</span> <span class="s1">'work item was attempted'</span> <span class="ow">in</span> <span class="n">msg</span><span class="p">:</span> |
| <span class="k">return</span> <span class="o">-</span><span class="mi">1</span> |
| <span class="k">elif</span> <span class="s1">'Traceback'</span> <span class="ow">in</span> <span class="n">msg</span><span class="p">:</span> |
| <span class="k">return</span> <span class="mi">1</span> |
| <span class="k">return</span> <span class="mi">0</span> |
| |
| <span class="k">if</span> <span class="n">duration</span><span class="p">:</span> |
| <span class="n">start_secs</span> <span class="o">=</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> |
| <span class="n">duration_secs</span> <span class="o">=</span> <span class="n">duration</span> <span class="o">//</span> <span class="mi">1000</span> |
| |
| <span class="n">job_id</span> <span class="o">=</span> <span class="n">result</span><span class="o">.</span><span class="n">job_id</span><span class="p">()</span> |
| <span class="k">while</span> <span class="kc">True</span><span class="p">:</span> |
| <span class="n">response</span> <span class="o">=</span> <span class="n">runner</span><span class="o">.</span><span class="n">dataflow_client</span><span class="o">.</span><span class="n">get_job</span><span class="p">(</span><span class="n">job_id</span><span class="p">)</span> |
| <span class="c1"># If get() is called very soon after Create() the response may not contain</span> |
| <span class="c1"># an initialized 'currentState' field.</span> |
| <span class="k">if</span> <span class="n">response</span><span class="o">.</span><span class="n">currentState</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">response</span><span class="o">.</span><span class="n">currentState</span> <span class="o">!=</span> <span class="n">last_job_state</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">'Job </span><span class="si">%s</span><span class="s1"> is in state </span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> <span class="n">job_id</span><span class="p">,</span> <span class="n">response</span><span class="o">.</span><span class="n">currentState</span><span class="p">)</span> |
| <span class="n">last_job_state</span> <span class="o">=</span> <span class="n">response</span><span class="o">.</span><span class="n">currentState</span> |
| <span class="k">if</span> <span class="nb">str</span><span class="p">(</span><span class="n">response</span><span class="o">.</span><span class="n">currentState</span><span class="p">)</span> <span class="o">!=</span> <span class="s1">'JOB_STATE_RUNNING'</span><span class="p">:</span> |
| <span class="c1"># Stop checking for new messages on timeout, explanatory</span> |
| <span class="c1"># message received, success, or a terminal job state caused</span> |
| <span class="c1"># by the user that therefore doesn't require explanation.</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">final_countdown_timer_secs</span> <span class="o"><=</span> <span class="mf">0.0</span> |
| <span class="ow">or</span> <span class="n">last_error_msg</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> |
| <span class="ow">or</span> <span class="nb">str</span><span class="p">(</span><span class="n">response</span><span class="o">.</span><span class="n">currentState</span><span class="p">)</span> <span class="o">==</span> <span class="s1">'JOB_STATE_DONE'</span> |
| <span class="ow">or</span> <span class="nb">str</span><span class="p">(</span><span class="n">response</span><span class="o">.</span><span class="n">currentState</span><span class="p">)</span> <span class="o">==</span> <span class="s1">'JOB_STATE_CANCELLED'</span> |
| <span class="ow">or</span> <span class="nb">str</span><span class="p">(</span><span class="n">response</span><span class="o">.</span><span class="n">currentState</span><span class="p">)</span> <span class="o">==</span> <span class="s1">'JOB_STATE_UPDATED'</span> |
| <span class="ow">or</span> <span class="nb">str</span><span class="p">(</span><span class="n">response</span><span class="o">.</span><span class="n">currentState</span><span class="p">)</span> <span class="o">==</span> <span class="s1">'JOB_STATE_DRAINED'</span><span class="p">):</span> |
| <span class="k">break</span> |
| |
| <span class="c1"># Check that job is in a post-preparation state before starting the</span> |
| <span class="c1"># final countdown.</span> |
| <span class="k">if</span> <span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">response</span><span class="o">.</span><span class="n">currentState</span><span class="p">)</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">(</span> |
| <span class="s1">'JOB_STATE_PENDING'</span><span class="p">,</span> <span class="s1">'JOB_STATE_QUEUED'</span><span class="p">)):</span> |
| <span class="c1"># The job has failed; ensure we see any final error messages.</span> |
| <span class="n">sleep_secs</span> <span class="o">=</span> <span class="mf">1.0</span> <span class="c1"># poll faster during the final countdown</span> |
| <span class="n">final_countdown_timer_secs</span> <span class="o">-=</span> <span class="n">sleep_secs</span> |
| |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="n">sleep_secs</span><span class="p">)</span> |
| |
| <span class="c1"># Get all messages since beginning of the job run or since last message.</span> |
| <span class="n">page_token</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">while</span> <span class="kc">True</span><span class="p">:</span> |
| <span class="n">messages</span><span class="p">,</span> <span class="n">page_token</span> <span class="o">=</span> <span class="n">runner</span><span class="o">.</span><span class="n">dataflow_client</span><span class="o">.</span><span class="n">list_messages</span><span class="p">(</span> |
| <span class="n">job_id</span><span class="p">,</span> <span class="n">page_token</span><span class="o">=</span><span class="n">page_token</span><span class="p">,</span> <span class="n">start_time</span><span class="o">=</span><span class="n">last_message_time</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">m</span> <span class="ow">in</span> <span class="n">messages</span><span class="p">:</span> |
| <span class="n">message</span> <span class="o">=</span> <span class="s1">'</span><span class="si">%s</span><span class="s1">: </span><span class="si">%s</span><span class="s1">: </span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">m</span><span class="o">.</span><span class="n">time</span><span class="p">,</span> <span class="n">m</span><span class="o">.</span><span class="n">messageImportance</span><span class="p">,</span> <span class="n">m</span><span class="o">.</span><span class="n">messageText</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">last_message_time</span> <span class="ow">or</span> <span class="n">m</span><span class="o">.</span><span class="n">time</span> <span class="o">></span> <span class="n">last_message_time</span><span class="p">:</span> |
| <span class="n">last_message_time</span> <span class="o">=</span> <span class="n">m</span><span class="o">.</span><span class="n">time</span> |
| <span class="n">current_seen_messages</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="n">message</span> <span class="ow">in</span> <span class="n">current_seen_messages</span><span class="p">:</span> |
| <span class="c1"># Skip the message if it has already been seen at the current</span> |
| <span class="c1"># time. This could be the case since the list_messages API is</span> |
| <span class="c1"># queried starting at last_message_time.</span> |
| <span class="k">continue</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">current_seen_messages</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">message</span><span class="p">)</span> |
| <span class="c1"># Skip empty messages.</span> |
| <span class="k">if</span> <span class="n">m</span><span class="o">.</span><span class="n">messageImportance</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">continue</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="n">message</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">str</span><span class="p">(</span><span class="n">m</span><span class="o">.</span><span class="n">messageImportance</span><span class="p">)</span> <span class="o">==</span> <span class="s1">'JOB_MESSAGE_ERROR'</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">rank_error</span><span class="p">(</span><span class="n">m</span><span class="o">.</span><span class="n">messageText</span><span class="p">)</span> <span class="o">>=</span> <span class="n">last_error_rank</span><span class="p">:</span> |
| <span class="n">last_error_rank</span> <span class="o">=</span> <span class="n">rank_error</span><span class="p">(</span><span class="n">m</span><span class="o">.</span><span class="n">messageText</span><span class="p">)</span> |
| <span class="n">last_error_msg</span> <span class="o">=</span> <span class="n">m</span><span class="o">.</span><span class="n">messageText</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">page_token</span><span class="p">:</span> |
| <span class="k">break</span> |
| |
| <span class="k">if</span> <span class="n">duration</span><span class="p">:</span> |
| <span class="n">passed_secs</span> <span class="o">=</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> <span class="o">-</span> <span class="n">start_secs</span> |
| <span class="k">if</span> <span class="n">passed_secs</span> <span class="o">></span> <span class="n">duration_secs</span><span class="p">:</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span><span class="s1">'Timing out on waiting for job </span><span class="si">%s</span><span class="s1"> after </span><span class="si">%d</span><span class="s1"> seconds'</span><span class="p">,</span> |
| <span class="n">job_id</span><span class="p">,</span> <span class="n">passed_secs</span><span class="p">)</span> |
| <span class="k">break</span> |
| |
| <span class="n">result</span><span class="o">.</span><span class="n">_job</span> <span class="o">=</span> <span class="n">response</span> |
| <span class="n">runner</span><span class="o">.</span><span class="n">last_error_msg</span> <span class="o">=</span> <span class="n">last_error_msg</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.group_by_key_input_visitor"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.group_by_key_input_visitor">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">group_by_key_input_visitor</span><span class="p">():</span> |
| <span class="c1"># Imported here to avoid circular dependencies.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PipelineVisitor</span> |
| |
| <span class="k">class</span> <span class="nc">GroupByKeyInputVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| <span class="sd">"""A visitor that replaces `Any` element type for input `PCollection` of</span> |
| <span class="sd"> a `GroupByKey` or `_GroupByKeyOnly` with a `KV` type.</span> |
| |
| <span class="sd"> TODO(BEAM-115): Once Python SDk is compatible with the new Runner API,</span> |
| <span class="sd"> we could directly replace the coder instead of mutating the element type.</span> |
| <span class="sd"> """</span> |
| |
| <span class="k">def</span> <span class="nf">enter_composite_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">visit_transform</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">visit_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">):</span> |
| <span class="c1"># Imported here to avoid circular dependencies.</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">GroupByKey</span><span class="p">,</span> <span class="n">_GroupByKeyOnly</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="p">(</span><span class="n">GroupByKey</span><span class="p">,</span> <span class="n">_GroupByKeyOnly</span><span class="p">)):</span> |
| <span class="n">pcoll</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> |
| <span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">coerce_to_kv_type</span><span class="p">(</span> |
| <span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">)</span> |
| <span class="n">key_type</span><span class="p">,</span> <span class="n">value_type</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span><span class="o">.</span><span class="n">tuple_types</span> |
| <span class="k">if</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">:</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.portability.fn_api_runner_transforms</span> <span class="k">import</span> \ |
| <span class="n">only_element</span> |
| <span class="n">key</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="kc">None</span> <span class="k">if</span> <span class="kc">None</span> <span class="ow">in</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">keys</span><span class="p">()</span> |
| <span class="k">else</span> <span class="n">only_element</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">keys</span><span class="p">()))</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">[</span><span class="n">key</span><span class="p">]</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span> |
| <span class="n">key_type</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">value_type</span><span class="p">]]</span> |
| |
| <span class="k">return</span> <span class="n">GroupByKeyInputVisitor</span><span class="p">()</span></div> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">_set_pdone_visitor</span><span class="p">(</span><span class="n">pipeline</span><span class="p">):</span> |
| <span class="c1"># Imported here to avoid circular dependencies.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PipelineVisitor</span> |
| |
| <span class="k">class</span> <span class="nc">SetPDoneVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| |
| <span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span> <span class="o">=</span> <span class="n">pipeline</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">_maybe_fix_output</span><span class="p">(</span><span class="n">transform_node</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">:</span> |
| <span class="n">pval</span> <span class="o">=</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PDone</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span> |
| <span class="n">pval</span><span class="o">.</span><span class="n">producer</span> <span class="o">=</span> <span class="n">transform_node</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span> <span class="o">=</span> <span class="p">{</span><span class="kc">None</span><span class="p">:</span> <span class="n">pval</span><span class="p">}</span> |
| |
| <span class="k">def</span> <span class="nf">enter_composite_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">):</span> |
| <span class="n">SetPDoneVisitor</span><span class="o">.</span><span class="n">_maybe_fix_output</span><span class="p">(</span><span class="n">transform_node</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">visit_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">):</span> |
| <span class="n">SetPDoneVisitor</span><span class="o">.</span><span class="n">_maybe_fix_output</span><span class="p">(</span><span class="n">transform_node</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">SetPDoneVisitor</span><span class="p">(</span><span class="n">pipeline</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DataflowRunner.side_input_visitor"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.side_input_visitor">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">side_input_visitor</span><span class="p">():</span> |
| <span class="c1"># Imported here to avoid circular dependencies.</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PipelineVisitor</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">ParDo</span> |
| |
| <span class="k">class</span> <span class="nc">SideInputVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| <span class="sd">"""Ensures input `PCollection` used as a side inputs has a `KV` type.</span> |
| |
| <span class="sd"> TODO(BEAM-115): Once Python SDK is compatible with the new Runner API,</span> |
| <span class="sd"> we could directly replace the coder instead of mutating the element type.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="nf">visit_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="n">ParDo</span><span class="p">):</span> |
| <span class="n">new_side_inputs</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">ix</span><span class="p">,</span> <span class="n">side_input</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">):</span> |
| <span class="n">access_pattern</span> <span class="o">=</span> <span class="n">side_input</span><span class="o">.</span><span class="n">_side_input_data</span><span class="p">()</span><span class="o">.</span><span class="n">access_pattern</span> |
| <span class="k">if</span> <span class="n">access_pattern</span> <span class="o">==</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">ITERABLE</span><span class="o">.</span><span class="n">urn</span><span class="p">:</span> |
| <span class="c1"># Add a map to ('', value) as Dataflow currently only handles</span> |
| <span class="c1"># keyed side inputs.</span> |
| <span class="n">pipeline</span> <span class="o">=</span> <span class="n">side_input</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">pipeline</span> |
| <span class="n">new_side_input</span> <span class="o">=</span> <span class="n">_DataflowIterableSideInput</span><span class="p">(</span><span class="n">side_input</span><span class="p">)</span> |
| <span class="n">new_side_input</span><span class="o">.</span><span class="n">pvalue</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">(</span> |
| <span class="n">pipeline</span><span class="p">,</span> |
| <span class="n">element_type</span><span class="o">=</span><span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span> |
| <span class="nb">bytes</span><span class="p">,</span> <span class="n">side_input</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">element_type</span><span class="p">])</span> |
| <span class="n">parent</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">parent</span> <span class="ow">or</span> <span class="n">pipeline</span><span class="o">.</span><span class="n">_root_transform</span><span class="p">()</span> |
| <span class="n">map_to_void_key</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">AppliedPTransform</span><span class="p">(</span> |
| <span class="n">pipeline</span><span class="p">,</span> |
| <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="sa">b</span><span class="s1">''</span><span class="p">,</span> <span class="n">x</span><span class="p">)),</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span> <span class="o">+</span> <span class="s1">'/MapToVoidKey</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="n">ix</span><span class="p">,</span> |
| <span class="p">(</span><span class="n">side_input</span><span class="o">.</span><span class="n">pvalue</span><span class="p">,))</span> |
| <span class="n">new_side_input</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">producer</span> <span class="o">=</span> <span class="n">map_to_void_key</span> |
| <span class="n">map_to_void_key</span><span class="o">.</span><span class="n">add_output</span><span class="p">(</span><span class="n">new_side_input</span><span class="o">.</span><span class="n">pvalue</span><span class="p">)</span> |
| <span class="n">parent</span><span class="o">.</span><span class="n">add_part</span><span class="p">(</span><span class="n">map_to_void_key</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">access_pattern</span> <span class="o">==</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">MULTIMAP</span><span class="o">.</span><span class="n">urn</span><span class="p">:</span> |
| <span class="c1"># Ensure the input coder is a KV coder and patch up the</span> |
| <span class="c1"># access pattern to appease Dataflow.</span> |
| <span class="n">side_input</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">coerce_to_kv_type</span><span class="p">(</span> |
| <span class="n">side_input</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">element_type</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">)</span> |
| <span class="n">new_side_input</span> <span class="o">=</span> <span class="n">_DataflowMultimapSideInput</span><span class="p">(</span><span class="n">side_input</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Unsupported access pattern for </span><span class="si">%r</span><span class="s1">: </span><span class="si">%r</span><span class="s1">'</span> <span class="o">%</span> |
| <span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">access_pattern</span><span class="p">))</span> |
| <span class="n">new_side_inputs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">new_side_input</span><span class="p">)</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">side_inputs</span> <span class="o">=</span> <span class="n">new_side_inputs</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">side_inputs</span> <span class="o">=</span> <span class="n">new_side_inputs</span> |
| |
| <span class="k">return</span> <span class="n">SideInputVisitor</span><span class="p">()</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.flatten_input_visitor"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.flatten_input_visitor">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">flatten_input_visitor</span><span class="p">():</span> |
| <span class="c1"># Imported here to avoid circular dependencies.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pipeline</span> <span class="k">import</span> <span class="n">PipelineVisitor</span> |
| |
| <span class="k">class</span> <span class="nc">FlattenInputVisitor</span><span class="p">(</span><span class="n">PipelineVisitor</span><span class="p">):</span> |
| <span class="sd">"""A visitor that replaces the element type for input ``PCollections``s of</span> |
| <span class="sd"> a ``Flatten`` transform with that of the output ``PCollection``.</span> |
| <span class="sd"> """</span> |
| |
| <span class="k">def</span> <span class="nf">visit_transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">):</span> |
| <span class="c1"># Imported here to avoid circular dependencies.</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">Flatten</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="p">,</span> <span class="n">Flatten</span><span class="p">):</span> |
| <span class="n">output_pcoll</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">[</span><span class="kc">None</span><span class="p">]</span> |
| <span class="k">for</span> <span class="n">input_pcoll</span> <span class="ow">in</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">:</span> |
| <span class="n">input_pcoll</span><span class="o">.</span><span class="n">element_type</span> <span class="o">=</span> <span class="n">output_pcoll</span><span class="o">.</span><span class="n">element_type</span> |
| |
| <span class="k">return</span> <span class="n">FlattenInputVisitor</span><span class="p">()</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.run_pipeline"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run_pipeline">[docs]</a> <span class="k">def</span> <span class="nf">run_pipeline</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="sd">"""Remotely executes entire pipeline or parts reachable from node."""</span> |
| <span class="c1"># Import here to avoid adding the dependency for local running scenarios.</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">apiclient</span> |
| <span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ImportError</span><span class="p">(</span> |
| <span class="s1">'Google Cloud Dataflow runner not available, '</span> |
| <span class="s1">'please install apache_beam[gcp]'</span><span class="p">)</span> |
| |
| <span class="c1"># Convert all side inputs into a form acceptable to Dataflow.</span> |
| <span class="k">if</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_fnapi</span><span class="p">(</span><span class="n">options</span><span class="p">):</span> |
| <span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">side_input_visitor</span><span class="p">())</span> |
| |
| <span class="c1"># Performing configured PTransform overrides. Note that this is currently</span> |
| <span class="c1"># done before Runner API serialization, since the new proto needs to contain</span> |
| <span class="c1"># any added PTransforms.</span> |
| <span class="n">pipeline</span><span class="o">.</span><span class="n">replace_all</span><span class="p">(</span><span class="n">DataflowRunner</span><span class="o">.</span><span class="n">_PTRANSFORM_OVERRIDES</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_sdf_bounded_source</span><span class="p">(</span><span class="n">options</span><span class="p">):</span> |
| <span class="n">pipeline</span><span class="o">.</span><span class="n">replace_all</span><span class="p">(</span><span class="n">DataflowRunner</span><span class="o">.</span><span class="n">_SDF_PTRANSFORM_OVERRIDES</span><span class="p">)</span> |
| |
| <span class="n">use_fnapi</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_fnapi</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="k">import</span> <span class="n">beam_runner_api_pb2</span> |
| <span class="n">default_container_image</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">apiclient</span><span class="o">.</span><span class="n">get_default_container_image_for_current_sdk</span><span class="p">(</span><span class="n">use_fnapi</span><span class="p">))</span> |
| <span class="n">default_environment</span> <span class="o">=</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">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">payload</span><span class="o">=</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="n">default_container_image</span><span class="p">)</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">())</span> |
| |
| <span class="c1"># Snapshot the pipeline in a portable proto.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">proto_pipeline</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">proto_context</span> <span class="o">=</span> <span class="n">pipeline</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">(</span> |
| <span class="n">return_context</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">default_environment</span><span class="o">=</span><span class="n">default_environment</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">use_fnapi</span><span class="p">:</span> |
| <span class="c1"># Cross language transform require using a pipeline object constructed</span> |
| <span class="c1"># from the full pipeline proto to make sure that expanded version of</span> |
| <span class="c1"># external transforms are reflected in the Pipeline job graph.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="k">import</span> <span class="n">Pipeline</span> |
| <span class="n">pipeline</span> <span class="o">=</span> <span class="n">Pipeline</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">proto_pipeline</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">.</span><span class="n">runner</span><span class="p">,</span> <span class="n">options</span><span class="p">,</span> |
| <span class="n">allow_proto_holders</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| |
| <span class="c1"># Pipelines generated from proto do not have output set to PDone set for</span> |
| <span class="c1"># leaf elements.</span> |
| <span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_set_pdone_visitor</span><span class="p">(</span><span class="n">pipeline</span><span class="p">))</span> |
| |
| <span class="c1"># We need to generate a new context that maps to the new pipeline object.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">proto_pipeline</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">proto_context</span> <span class="o">=</span> <span class="n">pipeline</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">(</span> |
| <span class="n">return_context</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">default_environment</span><span class="o">=</span><span class="n">default_environment</span><span class="p">)</span> |
| |
| <span class="c1"># Add setup_options for all the BeamPlugin imports</span> |
| <span class="n">setup_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">SetupOptions</span><span class="p">)</span> |
| <span class="n">plugins</span> <span class="o">=</span> <span class="n">BeamPlugin</span><span class="o">.</span><span class="n">get_all_plugin_paths</span><span class="p">()</span> |
| <span class="k">if</span> <span class="n">setup_options</span><span class="o">.</span><span class="n">beam_plugins</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">plugins</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">plugins</span> <span class="o">+</span> <span class="n">setup_options</span><span class="o">.</span><span class="n">beam_plugins</span><span class="p">))</span> |
| <span class="n">setup_options</span><span class="o">.</span><span class="n">beam_plugins</span> <span class="o">=</span> <span class="n">plugins</span> |
| |
| <span class="c1"># Elevate "min_cpu_platform" to pipeline option, but using the existing</span> |
| <span class="c1"># experiment.</span> |
| <span class="n">debug_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">DebugOptions</span><span class="p">)</span> |
| <span class="n">worker_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">WorkerOptions</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">worker_options</span><span class="o">.</span><span class="n">min_cpu_platform</span><span class="p">:</span> |
| <span class="n">debug_options</span><span class="o">.</span><span class="n">add_experiment</span><span class="p">(</span><span class="s1">'min_cpu_platform='</span> <span class="o">+</span> |
| <span class="n">worker_options</span><span class="o">.</span><span class="n">min_cpu_platform</span><span class="p">)</span> |
| |
| <span class="c1"># Elevate "enable_streaming_engine" to pipeline option, but using the</span> |
| <span class="c1"># existing experiment.</span> |
| <span class="n">google_cloud_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">GoogleCloudOptions</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">google_cloud_options</span><span class="o">.</span><span class="n">enable_streaming_engine</span><span class="p">:</span> |
| <span class="n">debug_options</span><span class="o">.</span><span class="n">add_experiment</span><span class="p">(</span><span class="s2">"enable_windmill_service"</span><span class="p">)</span> |
| <span class="n">debug_options</span><span class="o">.</span><span class="n">add_experiment</span><span class="p">(</span><span class="s2">"enable_streaming_engine"</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">debug_options</span><span class="o">.</span><span class="n">lookup_experiment</span><span class="p">(</span><span class="s2">"enable_windmill_service"</span><span class="p">)</span> <span class="ow">or</span> |
| <span class="n">debug_options</span><span class="o">.</span><span class="n">lookup_experiment</span><span class="p">(</span><span class="s2">"enable_streaming_engine"</span><span class="p">)):</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">"""Streaming engine both disabled and enabled:</span> |
| <span class="s2"> enable_streaming_engine flag is not set, but enable_windmill_service</span> |
| <span class="s2"> and/or enable_streaming_engine experiments are present.</span> |
| <span class="s2"> It is recommended you only set the enable_streaming_engine flag."""</span><span class="p">)</span> |
| |
| <span class="n">dataflow_worker_jar</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">worker_options</span><span class="p">,</span> <span class="s1">'dataflow_worker_jar'</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">dataflow_worker_jar</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_fnapi</span><span class="p">(</span><span class="n">options</span><span class="p">):</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span> |
| <span class="s1">'Typical end users should not use this worker jar feature. '</span> |
| <span class="s1">'It can only be used when FnAPI is enabled.'</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">debug_options</span><span class="o">.</span><span class="n">add_experiment</span><span class="p">(</span><span class="s1">'use_staged_dataflow_worker_jar'</span><span class="p">)</span> |
| |
| <span class="c1"># Make Dataflow workers use FastAvro on Python 3 unless use_avro experiment</span> |
| <span class="c1"># is set. Note that use_avro is only interpreted by the Dataflow runner</span> |
| <span class="c1"># at job submission and is not interpreted by Dataflow service or workers,</span> |
| <span class="c1"># which by default use avro library unless use_fastavro experiment is set.</span> |
| <span class="k">if</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="o">></span> <span class="mi">2</span> <span class="ow">and</span> <span class="p">(</span> |
| <span class="ow">not</span> <span class="n">debug_options</span><span class="o">.</span><span class="n">lookup_experiment</span><span class="p">(</span><span class="s1">'use_avro'</span><span class="p">)):</span> |
| <span class="n">debug_options</span><span class="o">.</span><span class="n">add_experiment</span><span class="p">(</span><span class="s1">'use_fastavro'</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">job</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">Job</span><span class="p">(</span><span class="n">options</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">proto_pipeline</span><span class="p">)</span> |
| |
| <span class="c1"># Dataflow runner requires a KV type for GBK inputs, hence we enforce that</span> |
| <span class="c1"># here.</span> |
| <span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">group_by_key_input_visitor</span><span class="p">())</span> |
| |
| <span class="c1"># Dataflow runner requires output type of the Flatten to be the same as the</span> |
| <span class="c1"># inputs, hence we enforce that here.</span> |
| <span class="n">pipeline</span><span class="o">.</span><span class="n">visit</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">flatten_input_visitor</span><span class="p">())</span> |
| |
| <span class="c1"># The superclass's run will trigger a traversal of all reachable nodes.</span> |
| <span class="nb">super</span><span class="p">(</span><span class="n">DataflowRunner</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="n">run_pipeline</span><span class="p">(</span><span class="n">pipeline</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span> |
| |
| <span class="n">test_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">TestOptions</span><span class="p">)</span> |
| <span class="c1"># If it is a dry run, return without submitting the job.</span> |
| <span class="k">if</span> <span class="n">test_options</span><span class="o">.</span><span class="n">dry_run</span><span class="p">:</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| |
| <span class="c1"># Get a Dataflow API client and set its options</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">dataflow_client</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">DataflowApplicationClient</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> |
| |
| <span class="c1"># Create the job description and send a request to the service. The result</span> |
| <span class="c1"># can be None if there is no need to send a request to the service (e.g.</span> |
| <span class="c1"># template creation). If a request was sent and failed then the call will</span> |
| <span class="c1"># raise an exception.</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">DataflowPipelineResult</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">dataflow_client</span><span class="o">.</span><span class="n">create_job</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">job</span><span class="p">),</span> <span class="bp">self</span><span class="p">)</span> |
| |
| <span class="c1"># TODO(BEAM-4274): Circular import runners-metrics. Requires refactoring.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.dataflow_metrics</span> <span class="k">import</span> <span class="n">DataflowMetrics</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_metrics</span> <span class="o">=</span> <span class="n">DataflowMetrics</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">dataflow_client</span><span class="p">,</span> <span class="n">result</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">job</span><span class="p">)</span> |
| <span class="n">result</span><span class="o">.</span><span class="n">metric_results</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_metrics</span> |
| <span class="k">return</span> <span class="n">result</span></div> |
| |
| <span class="k">def</span> <span class="nf">_get_typehint_based_encoding</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">typehint</span><span class="p">,</span> <span class="n">window_coder</span><span class="p">,</span> <span class="n">use_fnapi</span><span class="p">):</span> |
| <span class="sd">"""Returns an encoding based on a typehint object."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_cloud_encoding</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_get_coder</span><span class="p">(</span><span class="n">typehint</span><span class="p">,</span> <span class="n">window_coder</span><span class="o">=</span><span class="n">window_coder</span><span class="p">),</span> <span class="n">use_fnapi</span><span class="p">)</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">_get_coder</span><span class="p">(</span><span class="n">typehint</span><span class="p">,</span> <span class="n">window_coder</span><span class="p">):</span> |
| <span class="sd">"""Returns a coder based on a typehint object."""</span> |
| <span class="k">if</span> <span class="n">window_coder</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">coders</span><span class="o">.</span><span class="n">WindowedValueCoder</span><span class="p">(</span> |
| <span class="n">coders</span><span class="o">.</span><span class="n">registry</span><span class="o">.</span><span class="n">get_coder</span><span class="p">(</span><span class="n">typehint</span><span class="p">),</span> |
| <span class="n">window_coder</span><span class="o">=</span><span class="n">window_coder</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">coders</span><span class="o">.</span><span class="n">registry</span><span class="o">.</span><span class="n">get_coder</span><span class="p">(</span><span class="n">typehint</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_get_cloud_encoding</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">coder</span><span class="p">,</span> <span class="n">use_fnapi</span><span class="p">):</span> |
| <span class="sd">"""Returns an encoding based on a coder object."""</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">coder</span><span class="p">,</span> <span class="n">coders</span><span class="o">.</span><span class="n">Coder</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s1">'Coder object must inherit from coders.Coder: </span><span class="si">%s</span><span class="s1">.'</span> <span class="o">%</span> |
| <span class="nb">str</span><span class="p">(</span><span class="n">coder</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">coder</span><span class="o">.</span><span class="n">as_cloud_object</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">proto_context</span> |
| <span class="o">.</span><span class="n">coders</span> <span class="k">if</span> <span class="n">use_fnapi</span> <span class="k">else</span> <span class="kc">None</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_get_side_input_encoding</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">input_encoding</span><span class="p">):</span> |
| <span class="sd">"""Returns an encoding for the output of a view transform.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> input_encoding: encoding of current transform's input. Side inputs need</span> |
| <span class="sd"> this because the service will check that input and output types match.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> An encoding that matches the output and input encoding. This is essential</span> |
| <span class="sd"> for the View transforms introduced to produce side inputs to a ParDo.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="p">{</span> |
| <span class="s1">'@type'</span><span class="p">:</span> <span class="s1">'kind:stream'</span><span class="p">,</span> |
| <span class="s1">'component_encodings'</span><span class="p">:</span> <span class="p">[</span><span class="n">input_encoding</span><span class="p">],</span> |
| <span class="s1">'is_stream_like'</span><span class="p">:</span> <span class="p">{</span> |
| <span class="s1">'value'</span><span class="p">:</span> <span class="kc">True</span> |
| <span class="p">},</span> |
| <span class="p">}</span> |
| |
| <span class="k">def</span> <span class="nf">_get_encoded_output_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">window_value</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="sd">"""Returns the cloud encoding of the coder for the output of a transform."""</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.portability.fn_api_runner_transforms</span> <span class="k">import</span> \ |
| <span class="n">only_element</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="n">output_tag</span> <span class="o">=</span> <span class="n">only_element</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| <span class="c1"># TODO(robertwb): Handle type hints for multi-output transforms.</span> |
| <span class="n">element_type</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">[</span><span class="n">output_tag</span><span class="p">]</span><span class="o">.</span><span class="n">element_type</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># TODO(silviuc): Remove this branch (and assert) when typehints are</span> |
| <span class="c1"># propagated everywhere. Returning an 'Any' as type hint will trigger</span> |
| <span class="c1"># usage of the fallback coder (i.e., cPickler).</span> |
| <span class="n">element_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span> |
| <span class="k">if</span> <span class="n">window_value</span><span class="p">:</span> |
| <span class="c1"># All outputs have the same windowing. So getting the coder from an</span> |
| <span class="c1"># arbitrary window is fine.</span> |
| <span class="n">output_tag</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="nb">iter</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">keys</span><span class="p">()))</span> |
| <span class="n">window_coder</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">[</span> |
| <span class="n">output_tag</span><span class="p">]</span><span class="o">.</span><span class="n">windowing</span><span class="o">.</span><span class="n">windowfn</span><span class="o">.</span><span class="n">get_window_coder</span><span class="p">())</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">window_coder</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">apiclient</span> |
| <span class="n">use_fnapi</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_fnapi</span><span class="p">(</span> |
| <span class="nb">list</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">values</span><span class="p">())[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">_options</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_typehint_based_encoding</span><span class="p">(</span><span class="n">element_type</span><span class="p">,</span> <span class="n">window_coder</span><span class="p">,</span> |
| <span class="n">use_fnapi</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_add_step</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">step_kind</span><span class="p">,</span> <span class="n">step_label</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">side_tags</span><span class="o">=</span><span class="p">()):</span> |
| <span class="sd">"""Creates a Step object and adds it to the cache."""</span> |
| <span class="c1"># Import here to avoid adding the dependency for local running scenarios.</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">apiclient</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">Step</span><span class="p">(</span><span class="n">step_kind</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_unique_step_name</span><span class="p">())</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">job</span><span class="o">.</span><span class="n">proto</span><span class="o">.</span><span class="n">steps</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">step</span><span class="o">.</span><span class="n">proto</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">,</span> <span class="n">step_label</span><span class="p">)</span> |
| <span class="c1"># Cache the node/step association for the main output of the transform node.</span> |
| |
| <span class="c1"># Main output key of external transforms can be ambiguous, so we only tag if</span> |
| <span class="c1"># there's only one tag instead of None.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.portability.fn_api_runner_transforms</span> <span class="k">import</span> <span class="n">only_element</span> |
| <span class="n">output_tag</span> <span class="o">=</span> <span class="p">(</span><span class="n">only_element</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> <span class="o">==</span> <span class="mi">1</span> <span class="k">else</span> <span class="kc">None</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">cache_output</span><span class="p">(</span><span class="n">transform_node</span><span class="p">,</span> <span class="n">output_tag</span><span class="p">,</span> <span class="n">step</span><span class="p">)</span> |
| <span class="c1"># If side_tags is not () then this is a multi-output transform node and we</span> |
| <span class="c1"># need to cache the (node, tag, step) for each of the tags used to access</span> |
| <span class="c1"># the outputs. This is essential because the keys used to search in the</span> |
| <span class="c1"># cache always contain the tag.</span> |
| <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="n">side_tags</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">cache_output</span><span class="p">(</span><span class="n">transform_node</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">step</span><span class="p">)</span> |
| |
| <span class="c1"># Finally, we add the display data items to the pipeline step.</span> |
| <span class="c1"># If the transform contains no display data then an empty list is added.</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">DISPLAY_DATA</span><span class="p">,</span> |
| <span class="p">[</span><span class="n">item</span><span class="o">.</span><span class="n">get_dict</span><span class="p">()</span> <span class="k">for</span> <span class="n">item</span> <span class="ow">in</span> |
| <span class="n">DisplayData</span><span class="o">.</span><span class="n">create_from</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="p">)</span><span class="o">.</span><span class="n">items</span><span class="p">])</span> |
| |
| <span class="k">return</span> <span class="n">step</span> |
| |
| <span class="k">def</span> <span class="nf">_add_singleton_step</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">label</span><span class="p">,</span> <span class="n">full_label</span><span class="p">,</span> <span class="n">tag</span><span class="p">,</span> <span class="n">input_step</span><span class="p">,</span> <span class="n">windowing_strategy</span><span class="p">):</span> |
| <span class="sd">"""Creates a CollectionToSingleton step used to handle ParDo side inputs."""</span> |
| <span class="c1"># Import here to avoid adding the dependency for local running scenarios.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">apiclient</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">Step</span><span class="p">(</span><span class="n">TransformNames</span><span class="o">.</span><span class="n">COLLECTION_TO_SINGLETON</span><span class="p">,</span> <span class="n">label</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">job</span><span class="o">.</span><span class="n">proto</span><span class="o">.</span><span class="n">steps</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">step</span><span class="o">.</span><span class="n">proto</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">,</span> <span class="n">full_label</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">PARALLEL_INPUT</span><span class="p">,</span> |
| <span class="p">{</span><span class="s1">'@type'</span><span class="p">:</span> <span class="s1">'OutputReference'</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">STEP_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">proto</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">get_output</span><span class="p">(</span><span class="n">tag</span><span class="p">)})</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_side_input_encoding</span><span class="p">(</span><span class="n">input_step</span><span class="o">.</span><span class="n">encoding</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_INFO</span><span class="p">,</span> |
| <span class="p">[{</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">full_label</span><span class="p">,</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT</span><span class="p">)),</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">:</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">}])</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">WINDOWING_STRATEGY</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">serialize_windowing_strategy</span><span class="p">(</span><span class="n">windowing_strategy</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">step</span> |
| |
| <div class="viewcode-block" id="DataflowRunner.run_Impulse"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run_Impulse">[docs]</a> <span class="k">def</span> <span class="nf">run_Impulse</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="n">standard_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_add_step</span><span class="p">(</span> |
| <span class="n">TransformNames</span><span class="o">.</span><span class="n">READ</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">standard_options</span><span class="o">.</span><span class="n">streaming</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">FORMAT</span><span class="p">,</span> <span class="s1">'pubsub'</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_SUBSCRIPTION</span><span class="p">,</span> <span class="s1">'_starting_signal/'</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">FORMAT</span><span class="p">,</span> <span class="s1">'impulse'</span><span class="p">)</span> |
| <span class="n">encoded_impulse_element</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">WindowedValueCoder</span><span class="p">(</span> |
| <span class="n">coders</span><span class="o">.</span><span class="n">BytesCoder</span><span class="p">(),</span> |
| <span class="n">coders</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">GlobalWindowCoder</span><span class="p">())</span><span class="o">.</span><span class="n">get_impl</span><span class="p">()</span><span class="o">.</span><span class="n">encode_nested</span><span class="p">(</span> |
| <span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="o">.</span><span class="n">windowed_value</span><span class="p">(</span><span class="sa">b</span><span class="s1">''</span><span class="p">))</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">IMPULSE_ELEMENT</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">byte_array_to_json_string</span><span class="p">(</span><span class="n">encoded_impulse_element</span><span class="p">))</span> |
| |
| <span class="n">step</span><span class="o">.</span><span class="n">encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_encoded_output_coder</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_INFO</span><span class="p">,</span> |
| <span class="p">[{</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">)),</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">:</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">}])</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.run_Flatten"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run_Flatten">[docs]</a> <span class="k">def</span> <span class="nf">run_Flatten</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_add_step</span><span class="p">(</span><span class="n">TransformNames</span><span class="o">.</span><span class="n">FLATTEN</span><span class="p">,</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">)</span> |
| <span class="n">inputs</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">one_input</span> <span class="ow">in</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">:</span> |
| <span class="n">input_step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">get_pvalue</span><span class="p">(</span><span class="n">one_input</span><span class="p">)</span> |
| <span class="n">inputs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="p">{</span><span class="s1">'@type'</span><span class="p">:</span> <span class="s1">'OutputReference'</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">STEP_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">proto</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">get_output</span><span class="p">(</span><span class="n">one_input</span><span class="o">.</span><span class="n">tag</span><span class="p">)})</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">INPUTS</span><span class="p">,</span> <span class="n">inputs</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_encoded_output_coder</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_INFO</span><span class="p">,</span> |
| <span class="p">[{</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">)),</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">:</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">}])</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.apply_WriteToBigQuery"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.apply_WriteToBigQuery">[docs]</a> <span class="k">def</span> <span class="nf">apply_WriteToBigQuery</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="c1"># Make sure this is the WriteToBigQuery class that we expected, and that</span> |
| <span class="c1"># users did not specifically request the new BQ sink by passing experiment</span> |
| <span class="c1"># flag.</span> |
| |
| <span class="c1"># TODO(BEAM-6928): Remove this function for release 2.14.0.</span> |
| <span class="n">experiments</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">DebugOptions</span><span class="p">)</span><span class="o">.</span><span class="n">experiments</span> <span class="ow">or</span> <span class="p">[]</span> |
| <span class="k">if</span> <span class="p">(</span><span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">WriteToBigQuery</span><span class="p">)</span> |
| <span class="ow">or</span> <span class="s1">'use_beam_bq_sink'</span> <span class="ow">in</span> <span class="n">experiments</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">apply_PTransform</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">,</span> <span class="n">options</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">schema</span> <span class="o">==</span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">gcp</span><span class="o">.</span><span class="n">bigquery</span><span class="o">.</span><span class="n">SCHEMA_AUTODETECT</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span> |
| <span class="s1">'Schema auto-detection is not supported on the native sink'</span><span class="p">)</span> |
| <span class="n">standard_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">standard_options</span><span class="o">.</span><span class="n">streaming</span><span class="p">:</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">write_disposition</span> <span class="o">==</span> |
| <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">BigQueryDisposition</span><span class="o">.</span><span class="n">WRITE_TRUNCATE</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span><span class="s1">'Can not use write truncation mode in streaming'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">apply_PTransform</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">pcoll</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="kn">from</span> <span class="nn">apache_beam.io.gcp.bigquery_tools</span> <span class="k">import</span> <span class="n">parse_table_schema_from_json</span> |
| <span class="n">schema</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">schema</span><span class="p">:</span> |
| <span class="n">schema</span> <span class="o">=</span> <span class="n">parse_table_schema_from_json</span><span class="p">(</span><span class="n">json</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">schema</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">'WriteToBigQuery'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">Write</span><span class="p">(</span> |
| <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">BigQuerySink</span><span class="p">(</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">,</span> |
| <span class="n">schema</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">create_disposition</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">write_disposition</span><span class="p">,</span> |
| <span class="n">kms_key</span><span class="o">=</span><span class="n">transform</span><span class="o">.</span><span class="n">kms_key</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.apply_GroupByKey"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.apply_GroupByKey">[docs]</a> <span class="k">def</span> <span class="nf">apply_GroupByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="c1"># Infer coder of parent.</span> |
| <span class="c1">#</span> |
| <span class="c1"># TODO(ccy): make Coder inference and checking less specialized and more</span> |
| <span class="c1"># comprehensive.</span> |
| <span class="n">parent</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">producer</span> |
| <span class="k">if</span> <span class="n">parent</span><span class="p">:</span> |
| <span class="n">coder</span> <span class="o">=</span> <span class="n">parent</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">_infer_output_coder</span><span class="p">()</span> <span class="c1"># pylint: disable=protected-access</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">coder</span><span class="p">:</span> |
| <span class="n">coder</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_coder</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span> <span class="ow">or</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">coder</span><span class="o">.</span><span class="n">is_kv_coder</span><span class="p">():</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">((</span><span class="s1">'Coder for the GroupByKey operation "</span><span class="si">%s</span><span class="s1">" is not a '</span> |
| <span class="s1">'key-value coder: </span><span class="si">%s</span><span class="s1">.'</span><span class="p">)</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">label</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">))</span> |
| <span class="c1"># TODO(robertwb): Update the coder itself if it changed.</span> |
| <span class="n">coders</span><span class="o">.</span><span class="n">registry</span><span class="o">.</span><span class="n">verify_deterministic</span><span class="p">(</span> |
| <span class="n">coder</span><span class="o">.</span><span class="n">key_coder</span><span class="p">(),</span> <span class="s1">'GroupByKey operation "</span><span class="si">%s</span><span class="s1">"'</span> <span class="o">%</span> <span class="n">transform</span><span class="o">.</span><span class="n">label</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.run_GroupByKey"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run_GroupByKey">[docs]</a> <span class="k">def</span> <span class="nf">run_GroupByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="n">input_tag</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">tag</span> |
| <span class="n">input_step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">get_pvalue</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_add_step</span><span class="p">(</span> |
| <span class="n">TransformNames</span><span class="o">.</span><span class="n">GROUP</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">PARALLEL_INPUT</span><span class="p">,</span> |
| <span class="p">{</span><span class="s1">'@type'</span><span class="p">:</span> <span class="s1">'OutputReference'</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">STEP_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">proto</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">get_output</span><span class="p">(</span><span class="n">input_tag</span><span class="p">)})</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_encoded_output_coder</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_INFO</span><span class="p">,</span> |
| <span class="p">[{</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">)),</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">:</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">}])</span> |
| <span class="n">windowing</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">get_windowing</span><span class="p">(</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">SERIALIZED_FN</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">serialize_windowing_strategy</span><span class="p">(</span><span class="n">windowing</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.run_RunnerAPIPTransformHolder"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run_RunnerAPIPTransformHolder">[docs]</a> <span class="k">def</span> <span class="nf">run_RunnerAPIPTransformHolder</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="sd">"""Adding Dataflow runner job description for transform holder objects.</span> |
| |
| <span class="sd"> These holder transform objects are generated for some of the transforms that</span> |
| <span class="sd"> become available after a cross-language transform expansion, usually if the</span> |
| <span class="sd"> corresponding transform object cannot be generated in Python SDK (for</span> |
| <span class="sd"> example, a python `ParDo` transform cannot be generated without a serialized</span> |
| <span class="sd"> Python `DoFn` object).</span> |
| <span class="sd"> """</span> |
| <span class="n">urn</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">proto</span><span class="p">()</span><span class="o">.</span><span class="n">urn</span> |
| <span class="k">assert</span> <span class="n">urn</span> |
| <span class="c1"># TODO(chamikara): support other transforms that requires holder objects in</span> |
| <span class="c1"># Python SDk.</span> |
| <span class="k">if</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">PAR_DO</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span> <span class="n">urn</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">run_ParDo</span><span class="p">(</span><span class="n">transform_node</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="ne">NotImplementedError</span><span class="p">(</span><span class="n">urn</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.run_ParDo"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run_ParDo">[docs]</a> <span class="k">def</span> <span class="nf">run_ParDo</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="n">transform</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span> |
| <span class="n">input_tag</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">tag</span> |
| <span class="n">input_step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">get_pvalue</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> |
| |
| <span class="c1"># Attach side inputs.</span> |
| <span class="n">si_dict</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="n">si_labels</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="n">full_label_counts</span> <span class="o">=</span> <span class="n">defaultdict</span><span class="p">(</span><span class="nb">int</span><span class="p">)</span> |
| <span class="n">lookup_label</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">side_pval</span><span class="p">:</span> <span class="n">si_labels</span><span class="p">[</span><span class="n">side_pval</span><span class="p">]</span> |
| <span class="n">named_inputs</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">named_inputs</span><span class="p">()</span> |
| <span class="n">label_renames</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="k">for</span> <span class="n">ix</span><span class="p">,</span> <span class="n">side_pval</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">):</span> |
| <span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">side_pval</span><span class="p">,</span> <span class="n">AsSideInput</span><span class="p">)</span> |
| <span class="n">step_name</span> <span class="o">=</span> <span class="s1">'SideInput-'</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_unique_step_name</span><span class="p">()</span> |
| <span class="n">si_label</span> <span class="o">=</span> <span class="s1">'side</span><span class="si">%d</span><span class="s1">-</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">ix</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">)</span> |
| <span class="n">old_label</span> <span class="o">=</span> <span class="s1">'side</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="n">ix</span> |
| <span class="n">label_renames</span><span class="p">[</span><span class="n">old_label</span><span class="p">]</span> <span class="o">=</span> <span class="n">si_label</span> |
| <span class="k">assert</span> <span class="n">old_label</span> <span class="ow">in</span> <span class="n">named_inputs</span> |
| <span class="n">pcollection_label</span> <span class="o">=</span> <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="n">side_pval</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">producer</span><span class="o">.</span><span class="n">full_label</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">'/'</span><span class="p">)[</span><span class="o">-</span><span class="mi">1</span><span class="p">],</span> |
| <span class="n">side_pval</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">tag</span> <span class="k">if</span> <span class="n">side_pval</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">tag</span> <span class="k">else</span> <span class="s1">'out'</span><span class="p">)</span> |
| <span class="n">si_full_label</span> <span class="o">=</span> <span class="s1">'</span><span class="si">%s</span><span class="s1">/</span><span class="si">%s</span><span class="s1">(</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">)'</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> |
| <span class="n">side_pval</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span><span class="p">,</span> |
| <span class="n">pcollection_label</span><span class="p">,</span> |
| <span class="n">full_label_counts</span><span class="p">[</span><span class="n">pcollection_label</span><span class="p">])</span> |
| |
| <span class="c1"># Count the number of times the same PCollection is a side input</span> |
| <span class="c1"># to the same ParDo.</span> |
| <span class="n">full_label_counts</span><span class="p">[</span><span class="n">pcollection_label</span><span class="p">]</span> <span class="o">+=</span> <span class="mi">1</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_add_singleton_step</span><span class="p">(</span> |
| <span class="n">step_name</span><span class="p">,</span> <span class="n">si_full_label</span><span class="p">,</span> <span class="n">side_pval</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">tag</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">get_pvalue</span><span class="p">(</span><span class="n">side_pval</span><span class="o">.</span><span class="n">pvalue</span><span class="p">),</span> |
| <span class="n">side_pval</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">windowing</span><span class="p">)</span> |
| <span class="n">si_dict</span><span class="p">[</span><span class="n">si_label</span><span class="p">]</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="s1">'@type'</span><span class="p">:</span> <span class="s1">'OutputReference'</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">STEP_NAME</span><span class="p">:</span> <span class="n">step_name</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">}</span> |
| <span class="n">si_labels</span><span class="p">[</span><span class="n">side_pval</span><span class="p">]</span> <span class="o">=</span> <span class="n">si_label</span> |
| |
| <span class="c1"># Now create the step for the ParDo transform being handled.</span> |
| <span class="n">transform_name</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="o">.</span><span class="n">rsplit</span><span class="p">(</span><span class="s1">'/'</span><span class="p">,</span> <span class="mi">1</span><span class="p">)[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_add_step</span><span class="p">(</span> |
| <span class="n">TransformNames</span><span class="o">.</span><span class="n">DO</span><span class="p">,</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span> <span class="o">+</span> <span class="p">(</span> |
| <span class="s1">'/</span><span class="si">{}</span><span class="s1">'</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">transform_name</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">side_inputs</span> <span class="k">else</span> <span class="s1">''</span><span class="p">),</span> |
| <span class="n">transform_node</span><span class="p">,</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">output_tags</span><span class="p">)</span> |
| <span class="c1"># Import here to avoid adding the dependency for local running scenarios.</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">apiclient</span> |
| <span class="n">transform_proto</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">proto_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_proto</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| <span class="n">transform_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">proto_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| <span class="n">use_fnapi</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_fnapi</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> |
| <span class="n">use_unified_worker</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_unified_worker</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> |
| <span class="c1"># The data transmitted in SERIALIZED_FN is different depending on whether</span> |
| <span class="c1"># this is a fnapi pipeline or not.</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">use_fnapi</span> <span class="ow">and</span> |
| <span class="p">(</span><span class="n">transform_proto</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">PAR_DO</span><span class="o">.</span><span class="n">urn</span> <span class="ow">or</span> |
| <span class="n">use_unified_worker</span><span class="p">)):</span> |
| <span class="c1"># Patch side input ids to be unique across a given pipeline.</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">label_renames</span> <span class="ow">and</span> |
| <span class="n">transform_proto</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span> <span class="o">==</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">primitives</span><span class="o">.</span><span class="n">PAR_DO</span><span class="o">.</span><span class="n">urn</span><span class="p">):</span> |
| <span class="c1"># Patch PTransform proto.</span> |
| <span class="k">for</span> <span class="n">old</span><span class="p">,</span> <span class="n">new</span> <span class="ow">in</span> <span class="n">iteritems</span><span class="p">(</span><span class="n">label_renames</span><span class="p">):</span> |
| <span class="n">transform_proto</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="n">new</span><span class="p">]</span> <span class="o">=</span> <span class="n">transform_proto</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="n">old</span><span class="p">]</span> |
| <span class="k">del</span> <span class="n">transform_proto</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="n">old</span><span class="p">]</span> |
| |
| <span class="c1"># Patch ParDo proto.</span> |
| <span class="n">proto_type</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="o">.</span><span class="n">_known_urns</span><span class="p">[</span><span class="n">transform_proto</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">urn</span><span class="p">]</span> |
| <span class="n">proto</span> <span class="o">=</span> <span class="n">proto_utils</span><span class="o">.</span><span class="n">parse_Bytes</span><span class="p">(</span><span class="n">transform_proto</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">payload</span><span class="p">,</span> |
| <span class="n">proto_type</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">old</span><span class="p">,</span> <span class="n">new</span> <span class="ow">in</span> <span class="n">iteritems</span><span class="p">(</span><span class="n">label_renames</span><span class="p">):</span> |
| <span class="n">proto</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">[</span><span class="n">new</span><span class="p">]</span><span class="o">.</span><span class="n">CopyFrom</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">[</span><span class="n">old</span><span class="p">])</span> |
| <span class="k">del</span> <span class="n">proto</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">[</span><span class="n">old</span><span class="p">]</span> |
| <span class="n">transform_proto</span><span class="o">.</span><span class="n">spec</span><span class="o">.</span><span class="n">payload</span> <span class="o">=</span> <span class="n">proto</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">()</span> |
| <span class="c1"># We need to update the pipeline proto.</span> |
| <span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">proto_pipeline</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span><span class="p">[</span><span class="n">transform_id</span><span class="p">]</span> |
| <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">proto_pipeline</span><span class="o">.</span><span class="n">components</span><span class="o">.</span><span class="n">transforms</span><span class="p">[</span><span class="n">transform_id</span><span class="p">]</span> |
| <span class="o">.</span><span class="n">CopyFrom</span><span class="p">(</span><span class="n">transform_proto</span><span class="p">))</span> |
| <span class="n">serialized_data</span> <span class="o">=</span> <span class="n">transform_id</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">serialized_data</span> <span class="o">=</span> <span class="n">pickler</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pardo_fn_data</span><span class="p">(</span><span class="n">transform_node</span><span class="p">,</span> <span class="n">lookup_label</span><span class="p">))</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">SERIALIZED_FN</span><span class="p">,</span> <span class="n">serialized_data</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">PARALLEL_INPUT</span><span class="p">,</span> |
| <span class="p">{</span><span class="s1">'@type'</span><span class="p">:</span> <span class="s1">'OutputReference'</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">STEP_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">proto</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">get_output</span><span class="p">(</span><span class="n">input_tag</span><span class="p">)})</span> |
| <span class="c1"># Add side inputs if any.</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">NON_PARALLEL_INPUTS</span><span class="p">,</span> <span class="n">si_dict</span><span class="p">)</span> |
| |
| <span class="c1"># Generate description for the outputs. The output names</span> |
| <span class="c1"># will be 'out' for main output and 'out_<tag>' for a tagged output.</span> |
| <span class="c1"># Using 'out' as a tag will not clash with the name for main since it will</span> |
| <span class="c1"># be transformed into 'out_out' internally.</span> |
| <span class="n">outputs</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_encoded_output_coder</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| |
| <span class="c1"># Add the main output to the description.</span> |
| <span class="n">outputs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="p">{</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">)),</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">:</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">})</span> |
| <span class="k">for</span> <span class="n">side_tag</span> <span class="ow">in</span> <span class="n">transform</span><span class="o">.</span><span class="n">output_tags</span><span class="p">:</span> |
| <span class="c1"># The assumption here is that all outputs will have the same typehint</span> |
| <span class="c1"># and coder as the main output. This is certainly the case right now</span> |
| <span class="c1"># but conceivably it could change in the future.</span> |
| <span class="n">outputs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="p">{</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">side_tag</span><span class="p">)),</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">:</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">_</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">,</span> <span class="n">side_tag</span><span class="p">))})</span> |
| |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_INFO</span><span class="p">,</span> <span class="n">outputs</span><span class="p">)</span> |
| |
| <span class="c1"># Add the restriction encoding if we are a splittable DoFn</span> |
| <span class="c1"># and are using the Fn API on the unified worker.</span> |
| <span class="n">restriction_coder</span> <span class="o">=</span> <span class="n">transform</span><span class="o">.</span><span class="n">get_restriction_coder</span><span class="p">()</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">use_fnapi</span> <span class="ow">and</span> <span class="n">use_unified_worker</span> <span class="ow">and</span> <span class="n">restriction_coder</span><span class="p">):</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">RESTRICTION_ENCODING</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_get_cloud_encoding</span><span class="p">(</span> |
| <span class="n">restriction_coder</span><span class="p">,</span> <span class="n">use_fnapi</span><span class="p">))</span></div> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">_pardo_fn_data</span><span class="p">(</span><span class="n">transform_node</span><span class="p">,</span> <span class="n">get_label</span><span class="p">):</span> |
| <span class="n">transform</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span> |
| <span class="n">si_tags_and_types</span> <span class="o">=</span> <span class="p">[</span> <span class="c1"># pylint: disable=protected-access</span> |
| <span class="p">(</span><span class="n">get_label</span><span class="p">(</span><span class="n">side_pval</span><span class="p">),</span> <span class="n">side_pval</span><span class="o">.</span><span class="vm">__class__</span><span class="p">,</span> <span class="n">side_pval</span><span class="o">.</span><span class="n">_view_options</span><span class="p">())</span> |
| <span class="k">for</span> <span class="n">side_pval</span> <span class="ow">in</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">side_inputs</span><span class="p">]</span> |
| <span class="k">return</span> <span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">kwargs</span><span class="p">,</span> <span class="n">si_tags_and_types</span><span class="p">,</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">windowing</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DataflowRunner.apply_CombineValues"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.apply_CombineValues">[docs]</a> <span class="k">def</span> <span class="nf">apply_CombineValues</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.run_CombineValues"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run_CombineValues">[docs]</a> <span class="k">def</span> <span class="nf">run_CombineValues</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="n">transform</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span> |
| <span class="n">input_tag</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">tag</span> |
| <span class="n">input_step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">get_pvalue</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_add_step</span><span class="p">(</span> |
| <span class="n">TransformNames</span><span class="o">.</span><span class="n">COMBINE</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">)</span> |
| |
| <span class="c1"># The data transmitted in SERIALIZED_FN is different depending on whether</span> |
| <span class="c1"># this is a fnapi pipeline or not.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">apiclient</span> |
| <span class="n">use_fnapi</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_fnapi</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">use_fnapi</span><span class="p">:</span> |
| <span class="c1"># Fnapi pipelines send the transform ID of the CombineValues transform's</span> |
| <span class="c1"># parent composite because Dataflow expects the ID of a CombinePerKey</span> |
| <span class="c1"># transform.</span> |
| <span class="n">serialized_data</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">proto_context</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">get_id</span><span class="p">(</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">parent</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Combiner functions do not take deferred side-inputs (i.e. PValues) and</span> |
| <span class="c1"># therefore the code to handle extra args/kwargs is simpler than for the</span> |
| <span class="c1"># DoFn's of the ParDo transform. In the last, empty argument is where</span> |
| <span class="c1"># side inputs information would go.</span> |
| <span class="n">serialized_data</span> <span class="o">=</span> <span class="n">pickler</span><span class="o">.</span><span class="n">dumps</span><span class="p">((</span><span class="n">transform</span><span class="o">.</span><span class="n">fn</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">args</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">kwargs</span><span class="p">,</span> <span class="p">()))</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">SERIALIZED_FN</span><span class="p">,</span> <span class="n">serialized_data</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">PARALLEL_INPUT</span><span class="p">,</span> |
| <span class="p">{</span><span class="s1">'@type'</span><span class="p">:</span> <span class="s1">'OutputReference'</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">STEP_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">proto</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">get_output</span><span class="p">(</span><span class="n">input_tag</span><span class="p">)})</span> |
| <span class="c1"># Note that the accumulator must not have a WindowedValue encoding, while</span> |
| <span class="c1"># the output of this step does in fact have a WindowedValue encoding.</span> |
| <span class="n">accumulator_encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_cloud_encoding</span><span class="p">(</span> |
| <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span><span class="o">.</span><span class="n">fn</span><span class="o">.</span><span class="n">get_accumulator_coder</span><span class="p">(),</span> <span class="n">use_fnapi</span><span class="p">)</span> |
| <span class="n">output_encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_encoded_output_coder</span><span class="p">(</span><span class="n">transform_node</span><span class="p">)</span> |
| |
| <span class="n">step</span><span class="o">.</span><span class="n">encoding</span> <span class="o">=</span> <span class="n">output_encoding</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">,</span> <span class="n">accumulator_encoding</span><span class="p">)</span> |
| <span class="c1"># Generate description for main output 'out.'</span> |
| <span class="n">outputs</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="c1"># Add the main output to the description.</span> |
| <span class="n">outputs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="p">{</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">)),</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">:</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">})</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_INFO</span><span class="p">,</span> <span class="n">outputs</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.apply_Read"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.apply_Read">[docs]</a> <span class="k">def</span> <span class="nf">apply_Read</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform</span><span class="p">,</span> <span class="n">pbegin</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">,</span> <span class="s1">'format'</span><span class="p">):</span> |
| <span class="c1"># Consider native Read to be a primitive for dataflow.</span> |
| <span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">(</span><span class="n">pbegin</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">debug_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">DebugOptions</span><span class="p">)</span> |
| <span class="k">if</span> <span class="p">(</span> |
| <span class="n">debug_options</span><span class="o">.</span><span class="n">experiments</span> <span class="ow">and</span> |
| <span class="s1">'beam_fn_api'</span> <span class="ow">in</span> <span class="n">debug_options</span><span class="o">.</span><span class="n">experiments</span> |
| <span class="p">):</span> |
| <span class="c1"># Expand according to FnAPI primitives.</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">apply_PTransform</span><span class="p">(</span><span class="n">transform</span><span class="p">,</span> <span class="n">pbegin</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="c1"># Custom Read is also a primitive for non-FnAPI on dataflow.</span> |
| <span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">PCollection</span><span class="p">(</span><span class="n">pbegin</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.run_Read"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run_Read">[docs]</a> <span class="k">def</span> <span class="nf">run_Read</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="n">transform</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_add_step</span><span class="p">(</span> |
| <span class="n">TransformNames</span><span class="o">.</span><span class="n">READ</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">)</span> |
| <span class="c1"># TODO(mairbek): refactor if-else tree to use registerable functions.</span> |
| <span class="c1"># Initialize the source specific properties.</span> |
| |
| <span class="n">standard_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">,</span> <span class="s1">'format'</span><span class="p">):</span> |
| <span class="c1"># If a format is not set, we assume the source to be a custom source.</span> |
| <span class="n">source_dict</span> <span class="o">=</span> <span class="p">{}</span> |
| |
| <span class="n">source_dict</span><span class="p">[</span><span class="s1">'spec'</span><span class="p">]</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="s1">'@type'</span><span class="p">:</span> <span class="n">names</span><span class="o">.</span><span class="n">SOURCE_TYPE</span><span class="p">,</span> |
| <span class="n">names</span><span class="o">.</span><span class="n">SERIALIZED_SOURCE_KEY</span><span class="p">:</span> <span class="n">pickler</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">)</span> |
| <span class="p">}</span> |
| |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">source_dict</span><span class="p">[</span><span class="s1">'metadata'</span><span class="p">]</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="s1">'estimated_size_bytes'</span><span class="p">:</span> <span class="n">json_value</span><span class="o">.</span><span class="n">get_typed_value_descriptor</span><span class="p">(</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">estimate_size</span><span class="p">())</span> |
| <span class="p">}</span> |
| <span class="k">except</span> <span class="n">error</span><span class="o">.</span><span class="n">RuntimeValueProviderError</span><span class="p">:</span> |
| <span class="c1"># Size estimation is best effort, and this error is by value provider.</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span> |
| <span class="s1">'Could not estimate size of source </span><span class="si">%r</span><span class="s1"> due to '</span> <span class="o">+</span> \ |
| <span class="s1">'RuntimeValueProviderError'</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">)</span> |
| <span class="k">except</span> <span class="ne">Exception</span><span class="p">:</span> <span class="c1"># pylint: disable=broad-except</span> |
| <span class="c1"># Size estimation is best effort. So we log the error and continue.</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span> |
| <span class="s1">'Could not estimate size of source </span><span class="si">%r</span><span class="s1"> due to an exception: </span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">,</span> <span class="n">traceback</span><span class="o">.</span><span class="n">format_exc</span><span class="p">())</span> |
| |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">SOURCE_STEP_INPUT</span><span class="p">,</span> |
| <span class="n">source_dict</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">format</span> <span class="o">==</span> <span class="s1">'text'</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">FILE_PATTERN</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">path</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">format</span> <span class="o">==</span> <span class="s1">'bigquery'</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">standard_options</span><span class="o">.</span><span class="n">streaming</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">'BigQuery source is not currently available for use '</span> |
| <span class="s1">'in streaming pipelines.'</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_EXPORT_FORMAT</span><span class="p">,</span> <span class="s1">'FORMAT_AVRO'</span><span class="p">)</span> |
| <span class="c1"># TODO(silviuc): Add table validation if transform.source.validate.</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">table_reference</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_DATASET</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_TABLE</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">)</span> |
| <span class="c1"># If project owning the table was not specified then the project owning</span> |
| <span class="c1"># the workflow (current project) will be used.</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_PROJECT</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">query</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_QUERY</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">query</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_USE_LEGACY_SQL</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">use_legacy_sql</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_FLATTEN_RESULTS</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">flatten_results</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">'BigQuery source </span><span class="si">%r</span><span class="s1"> must specify either a table or'</span> |
| <span class="s1">' a query'</span> <span class="o">%</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">kms_key</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_KMS_KEY</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">kms_key</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">format</span> <span class="o">==</span> <span class="s1">'pubsub'</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">standard_options</span><span class="o">.</span><span class="n">streaming</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">'Cloud Pub/Sub is currently available for use '</span> |
| <span class="s1">'only in streaming pipelines.'</span><span class="p">)</span> |
| <span class="c1"># Only one of topic or subscription should be set.</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">full_subscription</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_SUBSCRIPTION</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">full_subscription</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">full_topic</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_TOPIC</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">full_topic</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">id_label</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_ID_LABEL</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">id_label</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">with_attributes</span><span class="p">:</span> |
| <span class="c1"># Setting this property signals Dataflow runner to return full</span> |
| <span class="c1"># PubsubMessages instead of just the data part of the payload.</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_SERIALIZED_ATTRIBUTES_FN</span><span class="p">,</span> <span class="s1">''</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">timestamp_attribute</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_TIMESTAMP_ATTRIBUTE</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">timestamp_attribute</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Source </span><span class="si">%r</span><span class="s1"> has unexpected format </span><span class="si">%s</span><span class="s1">.'</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">format</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="p">,</span> <span class="s1">'format'</span><span class="p">):</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">FORMAT</span><span class="p">,</span> <span class="n">names</span><span class="o">.</span><span class="n">SOURCE_FORMAT</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">FORMAT</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">format</span><span class="p">)</span> |
| |
| <span class="c1"># Wrap coder in WindowedValueCoder: this is necessary as the encoding of a</span> |
| <span class="c1"># step should be the type of value outputted by each step. Read steps</span> |
| <span class="c1"># automatically wrap output values in a WindowedValue wrapper, if necessary.</span> |
| <span class="c1"># This is also necessary for proper encoding for size estimation.</span> |
| <span class="c1"># Using a GlobalWindowCoder as a place holder instead of the default</span> |
| <span class="c1"># PickleCoder because GlobalWindowCoder is known coder.</span> |
| <span class="c1"># TODO(robertwb): Query the collection for the windowfn to extract the</span> |
| <span class="c1"># correct coder.</span> |
| <span class="n">coder</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">WindowedValueCoder</span><span class="p">(</span> |
| <span class="n">coders</span><span class="o">.</span><span class="n">registry</span><span class="o">.</span><span class="n">get_coder</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">outputs</span><span class="p">[</span><span class="kc">None</span><span class="p">]</span><span class="o">.</span><span class="n">element_type</span><span class="p">),</span> |
| <span class="n">coders</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">GlobalWindowCoder</span><span class="p">())</span> |
| |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">apiclient</span> |
| <span class="n">use_fnapi</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_fnapi</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_cloud_encoding</span><span class="p">(</span><span class="n">coder</span><span class="p">,</span> <span class="n">use_fnapi</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_INFO</span><span class="p">,</span> |
| <span class="p">[{</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">USER_NAME</span><span class="p">:</span> <span class="p">(</span> |
| <span class="s1">'</span><span class="si">%s</span><span class="s1">.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">)),</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">:</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUT</span><span class="p">}])</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.run__NativeWrite"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.run__NativeWrite">[docs]</a> <span class="k">def</span> <span class="nf">run__NativeWrite</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">,</span> <span class="n">options</span><span class="p">):</span> |
| <span class="n">transform</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">transform</span> |
| <span class="n">input_tag</span> <span class="o">=</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">tag</span> |
| <span class="n">input_step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cache</span><span class="o">.</span><span class="n">get_pvalue</span><span class="p">(</span><span class="n">transform_node</span><span class="o">.</span><span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_add_step</span><span class="p">(</span> |
| <span class="n">TransformNames</span><span class="o">.</span><span class="n">WRITE</span><span class="p">,</span> <span class="n">transform_node</span><span class="o">.</span><span class="n">full_label</span><span class="p">,</span> <span class="n">transform_node</span><span class="p">)</span> |
| <span class="c1"># TODO(mairbek): refactor if-else tree to use registerable functions.</span> |
| <span class="c1"># Initialize the sink specific properties.</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">format</span> <span class="o">==</span> <span class="s1">'text'</span><span class="p">:</span> |
| <span class="c1"># Note that it is important to use typed properties (@type/value dicts)</span> |
| <span class="c1"># for non-string properties and also for empty strings. For example,</span> |
| <span class="c1"># in the code below the num_shards must have type and also</span> |
| <span class="c1"># file_name_suffix and shard_name_template (could be empty strings).</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">FILE_NAME_PREFIX</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">file_name_prefix</span><span class="p">,</span> |
| <span class="n">with_type</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">FILE_NAME_SUFFIX</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">file_name_suffix</span><span class="p">,</span> |
| <span class="n">with_type</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">SHARD_NAME_TEMPLATE</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">shard_name_template</span><span class="p">,</span> |
| <span class="n">with_type</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">num_shards</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">NUM_SHARDS</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">num_shards</span><span class="p">,</span> <span class="n">with_type</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="c1"># TODO(silviuc): Implement sink validation.</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">VALIDATE_SINK</span><span class="p">,</span> <span class="kc">False</span><span class="p">,</span> <span class="n">with_type</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">format</span> <span class="o">==</span> <span class="s1">'bigquery'</span><span class="p">:</span> |
| <span class="c1"># TODO(silviuc): Add table validation if transform.sink.validate.</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_DATASET</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">datasetId</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_TABLE</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">tableId</span><span class="p">)</span> |
| <span class="c1"># If project owning the table was not specified then the project owning</span> |
| <span class="c1"># the workflow (current project) will be used.</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_PROJECT</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_reference</span><span class="o">.</span><span class="n">projectId</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_CREATE_DISPOSITION</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">create_disposition</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_WRITE_DISPOSITION</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">write_disposition</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">table_schema</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_SCHEMA</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">schema_as_json</span><span class="p">())</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">kms_key</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">BIGQUERY_KMS_KEY</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">kms_key</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">format</span> <span class="o">==</span> <span class="s1">'pubsub'</span><span class="p">:</span> |
| <span class="n">standard_options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">StandardOptions</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">standard_options</span><span class="o">.</span><span class="n">streaming</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">'Cloud Pub/Sub is currently available for use '</span> |
| <span class="s1">'only in streaming pipelines.'</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_TOPIC</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">full_topic</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">id_label</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_ID_LABEL</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">id_label</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">with_attributes</span><span class="p">:</span> |
| <span class="c1"># Setting this property signals Dataflow runner that the PCollection</span> |
| <span class="c1"># contains PubsubMessage objects instead of just raw data.</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_SERIALIZED_ATTRIBUTES_FN</span><span class="p">,</span> <span class="s1">''</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">timestamp_attribute</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">PUBSUB_TIMESTAMP_ATTRIBUTE</span><span class="p">,</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">timestamp_attribute</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Sink </span><span class="si">%r</span><span class="s1"> has unexpected format </span><span class="si">%s</span><span class="s1">.'</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">format</span><span class="p">))</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">FORMAT</span><span class="p">,</span> <span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">format</span><span class="p">)</span> |
| |
| <span class="c1"># Wrap coder in WindowedValueCoder: this is necessary for proper encoding</span> |
| <span class="c1"># for size estimation. Using a GlobalWindowCoder as a place holder instead</span> |
| <span class="c1"># of the default PickleCoder because GlobalWindowCoder is known coder.</span> |
| <span class="c1"># TODO(robertwb): Query the collection for the windowfn to extract the</span> |
| <span class="c1"># correct coder.</span> |
| <span class="n">coder</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">WindowedValueCoder</span><span class="p">(</span><span class="n">transform</span><span class="o">.</span><span class="n">sink</span><span class="o">.</span><span class="n">coder</span><span class="p">,</span> |
| <span class="n">coders</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">GlobalWindowCoder</span><span class="p">())</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.dataflow.internal</span> <span class="k">import</span> <span class="n">apiclient</span> |
| <span class="n">use_fnapi</span> <span class="o">=</span> <span class="n">apiclient</span><span class="o">.</span><span class="n">_use_fnapi</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">encoding</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_cloud_encoding</span><span class="p">(</span><span class="n">coder</span><span class="p">,</span> <span class="n">use_fnapi</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span><span class="n">PropertyNames</span><span class="o">.</span><span class="n">ENCODING</span><span class="p">,</span> <span class="n">step</span><span class="o">.</span><span class="n">encoding</span><span class="p">)</span> |
| <span class="n">step</span><span class="o">.</span><span class="n">add_property</span><span class="p">(</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">PARALLEL_INPUT</span><span class="p">,</span> |
| <span class="p">{</span><span class="s1">'@type'</span><span class="p">:</span> <span class="s1">'OutputReference'</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">STEP_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">proto</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> |
| <span class="n">PropertyNames</span><span class="o">.</span><span class="n">OUTPUT_NAME</span><span class="p">:</span> <span class="n">input_step</span><span class="o">.</span><span class="n">get_output</span><span class="p">(</span><span class="n">input_tag</span><span class="p">)})</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.serialize_windowing_strategy"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.serialize_windowing_strategy">[docs]</a> <span class="nd">@classmethod</span> |
| <span class="k">def</span> <span class="nf">serialize_windowing_strategy</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">windowing</span><span class="p">):</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners</span> <span class="k">import</span> <span class="n">pipeline_context</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="k">import</span> <span class="n">beam_runner_api_pb2</span> |
| <span class="n">context</span> <span class="o">=</span> <span class="n">pipeline_context</span><span class="o">.</span><span class="n">PipelineContext</span><span class="p">()</span> |
| <span class="n">windowing_proto</span> <span class="o">=</span> <span class="n">windowing</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">(</span><span class="n">context</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">cls</span><span class="o">.</span><span class="n">byte_array_to_json_string</span><span class="p">(</span> |
| <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">MessageWithComponents</span><span class="p">(</span> |
| <span class="n">components</span><span class="o">=</span><span class="n">context</span><span class="o">.</span><span class="n">to_runner_api</span><span class="p">(),</span> |
| <span class="n">windowing_strategy</span><span class="o">=</span><span class="n">windowing_proto</span><span class="p">)</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">())</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.deserialize_windowing_strategy"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.deserialize_windowing_strategy">[docs]</a> <span class="nd">@classmethod</span> |
| <span class="k">def</span> <span class="nf">deserialize_windowing_strategy</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">serialized_data</span><span class="p">):</span> |
| <span class="c1"># Imported here to avoid circular dependencies.</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners</span> <span class="k">import</span> <span class="n">pipeline_context</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="k">import</span> <span class="n">beam_runner_api_pb2</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="k">import</span> <span class="n">Windowing</span> |
| <span class="n">proto</span> <span class="o">=</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">MessageWithComponents</span><span class="p">()</span> |
| <span class="n">proto</span><span class="o">.</span><span class="n">ParseFromString</span><span class="p">(</span><span class="bp">cls</span><span class="o">.</span><span class="n">json_string_to_byte_array</span><span class="p">(</span><span class="n">serialized_data</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">Windowing</span><span class="o">.</span><span class="n">from_runner_api</span><span class="p">(</span> |
| <span class="n">proto</span><span class="o">.</span><span class="n">windowing_strategy</span><span class="p">,</span> |
| <span class="n">pipeline_context</span><span class="o">.</span><span class="n">PipelineContext</span><span class="p">(</span><span class="n">proto</span><span class="o">.</span><span class="n">components</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.byte_array_to_json_string"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.byte_array_to_json_string">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">byte_array_to_json_string</span><span class="p">(</span><span class="n">raw_bytes</span><span class="p">):</span> |
| <span class="sd">"""Implements org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString."""</span> |
| <span class="k">return</span> <span class="n">quote</span><span class="p">(</span><span class="n">raw_bytes</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DataflowRunner.json_string_to_byte_array"><a class="viewcode-back" href="../../../../apache_beam.runners.dataflow.dataflow_runner.html#apache_beam.runners.dataflow.dataflow_runner.DataflowRunner.json_string_to_byte_array">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">json_string_to_byte_array</span><span class="p">(</span><span class="n">encoded_string</span><span class="p">):</span> |
| <span class="sd">"""Implements org.apache.beam.sdk.util.StringUtils.jsonStringToByteArray."""</span> |
| <span class="k">return</span> <span class="n">unquote_to_bytes</span><span class="p">(</span><span class="n">encoded_string</span><span class="p">)</span></div></div> |
| |
| |
| <span class="k">class</span> <span class="nc">_DataflowSideInput</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">AsSideInput</span><span class="p">):</span> |
| <span class="sd">"""Wraps a side input as a dataflow-compatible side input."""</span> |
| |
| <span class="k">def</span> <span class="nf">_view_options</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">{</span> |
| <span class="s1">'data'</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">,</span> |
| <span class="p">}</span> |
| |
| <span class="k">def</span> <span class="nf">_side_input_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_data</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_DataflowIterableSideInput</span><span class="p">(</span><span class="n">_DataflowSideInput</span><span class="p">):</span> |
| <span class="sd">"""Wraps an iterable side input as dataflow-compatible side input."""</span> |
| |
| <span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">iterable_side_input</span><span class="p">):</span> |
| <span class="c1"># pylint: disable=protected-access</span> |
| <span class="n">side_input_data</span> <span class="o">=</span> <span class="n">iterable_side_input</span><span class="o">.</span><span class="n">_side_input_data</span><span class="p">()</span> |
| <span class="k">assert</span> <span class="p">(</span> |
| <span class="n">side_input_data</span><span class="o">.</span><span class="n">access_pattern</span> <span class="o">==</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">ITERABLE</span><span class="o">.</span><span class="n">urn</span><span class="p">)</span> |
| <span class="n">iterable_view_fn</span> <span class="o">=</span> <span class="n">side_input_data</span><span class="o">.</span><span class="n">view_fn</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_data</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">SideInputData</span><span class="p">(</span> |
| <span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">MULTIMAP</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> |
| <span class="n">side_input_data</span><span class="o">.</span><span class="n">window_mapping_fn</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">multimap</span><span class="p">:</span> <span class="n">iterable_view_fn</span><span class="p">(</span><span class="n">multimap</span><span class="p">[</span><span class="sa">b</span><span class="s1">''</span><span class="p">]))</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_DataflowMultimapSideInput</span><span class="p">(</span><span class="n">_DataflowSideInput</span><span class="p">):</span> |
| <span class="sd">"""Wraps a multimap side input as dataflow-compatible side input."""</span> |
| |
| <span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">side_input</span><span class="p">):</span> |
| <span class="c1"># pylint: disable=protected-access</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">pvalue</span> <span class="o">=</span> <span class="n">side_input</span><span class="o">.</span><span class="n">pvalue</span> |
| <span class="n">side_input_data</span> <span class="o">=</span> <span class="n">side_input</span><span class="o">.</span><span class="n">_side_input_data</span><span class="p">()</span> |
| <span class="k">assert</span> <span class="p">(</span> |
| <span class="n">side_input_data</span><span class="o">.</span><span class="n">access_pattern</span> <span class="o">==</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">MULTIMAP</span><span class="o">.</span><span class="n">urn</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_data</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">SideInputData</span><span class="p">(</span> |
| <span class="n">common_urns</span><span class="o">.</span><span class="n">side_inputs</span><span class="o">.</span><span class="n">MULTIMAP</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> |
| <span class="n">side_input_data</span><span class="o">.</span><span class="n">window_mapping_fn</span><span class="p">,</span> |
| <span class="n">side_input_data</span><span class="o">.</span><span class="n">view_fn</span><span class="p">)</span> |
| |
| |
| <span class="k">class</span> <span class="nc">DataflowPipelineResult</span><span class="p">(</span><span class="n">PipelineResult</span><span class="p">):</span> |
| <span class="sd">"""Represents the state of a pipeline run on the Dataflow service."""</span> |
| |
| <span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">job</span><span class="p">,</span> <span class="n">runner</span><span class="p">):</span> |
| <span class="sd">"""Initialize a new DataflowPipelineResult instance.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> job: Job message from the Dataflow API. Could be :data:`None` if a job</span> |
| <span class="sd"> request was not sent to Dataflow service (e.g. template jobs).</span> |
| <span class="sd"> runner: DataflowRunner instance.</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_job</span> <span class="o">=</span> <span class="n">job</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_runner</span> <span class="o">=</span> <span class="n">runner</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">metric_results</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="k">def</span> <span class="nf">_update_job</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># We need the job id to be able to update job information. There is no need</span> |
| <span class="c1"># to update the job if we are in a known terminal state.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">has_job</span> <span class="ow">and</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_in_terminal_state</span><span class="p">():</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_job</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_runner</span><span class="o">.</span><span class="n">dataflow_client</span><span class="o">.</span><span class="n">get_job</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">job_id</span><span class="p">())</span> |
| |
| <span class="k">def</span> <span class="nf">job_id</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_job</span><span class="o">.</span><span class="n">id</span> |
| |
| <span class="k">def</span> <span class="nf">metrics</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">metric_results</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">has_job</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_job</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> |
| |
| <span class="k">def</span> <span class="nf">_get_job_state</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="n">values_enum</span> <span class="o">=</span> <span class="n">dataflow_api</span><span class="o">.</span><span class="n">Job</span><span class="o">.</span><span class="n">CurrentStateValueValuesEnum</span> |
| |
| <span class="c1"># Ordered by the enum values. Values that may be introduced in</span> |
| <span class="c1"># future versions of Dataflow API are considered UNKNOWN by the SDK.</span> |
| <span class="n">api_jobstate_map</span> <span class="o">=</span> <span class="n">defaultdict</span><span class="p">(</span><span class="k">lambda</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">UNKNOWN</span><span class="p">,</span> <span class="p">{</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_UNKNOWN</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">UNKNOWN</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_STOPPED</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">STOPPED</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_RUNNING</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">RUNNING</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_DONE</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">DONE</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_FAILED</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">FAILED</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_CANCELLED</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">CANCELLED</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_UPDATED</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">UPDATED</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_DRAINING</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">DRAINING</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_DRAINED</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">DRAINED</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_PENDING</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">PENDING</span><span class="p">,</span> |
| <span class="n">values_enum</span><span class="o">.</span><span class="n">JOB_STATE_CANCELLING</span><span class="p">:</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">CANCELLING</span><span class="p">,</span> |
| <span class="p">})</span> |
| |
| <span class="k">return</span> <span class="p">(</span><span class="n">api_jobstate_map</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_job</span><span class="o">.</span><span class="n">currentState</span><span class="p">]</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_job</span><span class="o">.</span><span class="n">currentState</span> |
| <span class="k">else</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">UNKNOWN</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">state</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="sd">"""Return the current state of the remote job.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> A PipelineState object.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">has_job</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">UNKNOWN</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_update_job</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_job_state</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="nf">is_in_terminal_state</span><span class="p">(</span><span class="bp">self</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">has_job</span><span class="p">:</span> |
| <span class="k">return</span> <span class="kc">True</span> |
| |
| <span class="k">return</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">is_terminal</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_get_job_state</span><span class="p">())</span> |
| |
| <span class="k">def</span> <span class="nf">wait_until_finish</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">duration</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_in_terminal_state</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">has_job</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">IOError</span><span class="p">(</span><span class="s1">'Failed to get the Dataflow job id.'</span><span class="p">)</span> |
| |
| <span class="n">thread</span> <span class="o">=</span> <span class="n">threading</span><span class="o">.</span><span class="n">Thread</span><span class="p">(</span> |
| <span class="n">target</span><span class="o">=</span><span class="n">DataflowRunner</span><span class="o">.</span><span class="n">poll_for_job_completion</span><span class="p">,</span> |
| <span class="n">args</span><span class="o">=</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_runner</span><span class="p">,</span> <span class="bp">self</span><span class="p">,</span> <span class="n">duration</span><span class="p">))</span> |
| |
| <span class="c1"># Mark the thread as a daemon thread so a keyboard interrupt on the main</span> |
| <span class="c1"># thread will terminate everything. This is also the reason we will not</span> |
| <span class="c1"># use thread.join() to wait for the polling thread.</span> |
| <span class="n">thread</span><span class="o">.</span><span class="n">daemon</span> <span class="o">=</span> <span class="kc">True</span> |
| <span class="n">thread</span><span class="o">.</span><span class="n">start</span><span class="p">()</span> |
| <span class="k">while</span> <span class="n">thread</span><span class="o">.</span><span class="n">isAlive</span><span class="p">():</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="mf">5.0</span><span class="p">)</span> |
| |
| <span class="c1"># TODO: Merge the termination code in poll_for_job_completion and</span> |
| <span class="c1"># is_in_terminal_state.</span> |
| <span class="n">terminated</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_in_terminal_state</span><span class="p">()</span> |
| <span class="k">assert</span> <span class="n">duration</span> <span class="ow">or</span> <span class="n">terminated</span><span class="p">,</span> <span class="p">(</span> |
| <span class="s1">'Job did not reach to a terminal state after waiting indefinitely.'</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">terminated</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span> <span class="o">!=</span> <span class="n">PipelineState</span><span class="o">.</span><span class="n">DONE</span><span class="p">:</span> |
| <span class="c1"># TODO(BEAM-1290): Consider converting this to an error log based on</span> |
| <span class="c1"># theresolution of the issue.</span> |
| <span class="k">raise</span> <span class="n">DataflowRuntimeException</span><span class="p">(</span> |
| <span class="s1">'Dataflow pipeline failed. State: </span><span class="si">%s</span><span class="s1">, Error:</span><span class="se">\n</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> |
| <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">,</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_runner</span><span class="p">,</span> <span class="s1">'last_error_msg'</span><span class="p">,</span> <span class="kc">None</span><span class="p">)),</span> <span class="bp">self</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span> |
| |
| <span class="k">def</span> <span class="nf">cancel</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">has_job</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">IOError</span><span class="p">(</span><span class="s1">'Failed to get the Dataflow job id.'</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_update_job</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_in_terminal_state</span><span class="p">():</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span> |
| <span class="s1">'Cancel failed because job </span><span class="si">%s</span><span class="s1"> is already terminated in state </span><span class="si">%s</span><span class="s1">.'</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">job_id</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">)</span> |
| <span class="k">else</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">_runner</span><span class="o">.</span><span class="n">dataflow_client</span><span class="o">.</span><span class="n">modify_job_state</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">job_id</span><span class="p">(),</span> <span class="s1">'JOB_STATE_CANCELLED'</span><span class="p">):</span> |
| <span class="n">cancel_failed_message</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s1">'Failed to cancel job </span><span class="si">%s</span><span class="s1">, please go to the Developers Console to '</span> |
| <span class="s1">'cancel it manually.'</span><span class="p">)</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">job_id</span><span class="p">()</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">error</span><span class="p">(</span><span class="n">cancel_failed_message</span><span class="p">)</span> |
| <span class="k">raise</span> <span class="n">DataflowRuntimeException</span><span class="p">(</span><span class="n">cancel_failed_message</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">state</span> |
| |
| <span class="k">def</span> <span class="nf">__str__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="s1">'<</span><span class="si">%s</span><span class="s1"> </span><span class="si">%s</span><span class="s1"> </span><span class="si">%s</span><span class="s1">>'</span> <span class="o">%</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="vm">__name__</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">job_id</span><span class="p">(),</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">state</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="s1">'<</span><span class="si">%s</span><span class="s1"> </span><span class="si">%s</span><span class="s1"> at </span><span class="si">%s</span><span class="s1">>'</span> <span class="o">%</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="vm">__name__</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_job</span><span class="p">,</span> <span class="nb">hex</span><span class="p">(</span><span class="nb">id</span><span class="p">(</span><span class="bp">self</span><span class="p">)))</span> |
| |
| |
| <span class="k">class</span> <span class="nc">DataflowRuntimeException</span><span class="p">(</span><span class="ne">Exception</span><span class="p">):</span> |
| <span class="sd">"""Indicates an error has occurred in running this pipeline."""</span> |
| |
| <span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">msg</span><span class="p">,</span> <span class="n">result</span><span class="p">):</span> |
| <span class="nb">super</span><span class="p">(</span><span class="n">DataflowRuntimeException</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="n">msg</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">result</span> <span class="o">=</span> <span class="n">result</span> |
| </pre></div> |
| |
| </div> |
| <div class="articleComments"> |
| |
| </div> |
| </div> |
| <footer> |
| |
| |
| <hr/> |
| |
| <div role="contentinfo"> |
| <p> |
| © Copyright . |
| |
| </p> |
| </div> |
| Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>. |
| |
| </footer> |
| |
| </div> |
| </div> |
| |
| </section> |
| |
| </div> |
| |
| |
| |
| |
| |
| <script type="text/javascript"> |
| var DOCUMENTATION_OPTIONS = { |
| URL_ROOT:'../../../../', |
| VERSION:'', |
| COLLAPSE_INDEX:false, |
| FILE_SUFFIX:'.html', |
| HAS_SOURCE: true, |
| SOURCELINK_SUFFIX: '.txt' |
| }; |
| </script> |
| <script type="text/javascript" src="../../../../_static/jquery.js"></script> |
| <script type="text/javascript" src="../../../../_static/underscore.js"></script> |
| <script type="text/javascript" src="../../../../_static/doctools.js"></script> |
| |
| |
| |
| |
| |
| <script type="text/javascript" src="../../../../_static/js/theme.js"></script> |
| |
| |
| |
| |
| <script type="text/javascript"> |
| jQuery(function () { |
| SphinxRtdTheme.StickyNav.enable(); |
| }); |
| </script> |
| |
| |
| </body> |
| </html> |