blob: 742ecffeafde9a439f85664a18b3c02ffd5eaed1 [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.ml.inference.utils &mdash; Apache Beam 2.47.0 documentation</title>
<script type="text/javascript" src="../../../../_static/js/modernizr.min.js"></script>
<script type="text/javascript" id="documentation_options" data-url_root="../../../../" src="../../../../_static/documentation_options.js"></script>
<script type="text/javascript" src="../../../../_static/jquery.js"></script>
<script type="text/javascript" src="../../../../_static/underscore.js"></script>
<script type="text/javascript" src="../../../../_static/doctools.js"></script>
<script type="text/javascript" src="../../../../_static/language_data.js"></script>
<script async="async" type="text/javascript" src="https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.5/latest.js?config=TeX-AMS-MML_HTMLorMML"></script>
<script type="text/javascript" src="../../../../_static/js/theme.js"></script>
<link rel="stylesheet" href="../../../../_static/css/theme.css" type="text/css" />
<link rel="stylesheet" href="../../../../_static/pygments.css" type="text/css" />
<link rel="index" title="Index" href="../../../../genindex.html" />
<link rel="search" title="Search" href="../../../../search.html" />
</head>
<body class="wy-body-for-nav">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search" >
<a href="../../../../index.html" class="icon icon-home"> Apache Beam
</a>
<div class="version">
2.47.0
</div>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.dataframe.html">apache_beam.dataframe package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.ml.html">apache_beam.ml package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.utils.html">apache_beam.utils package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.yaml.html">apache_beam.yaml package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../../index.html">Docs</a> &raquo;</li>
<li><a href="../../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.ml.inference.utils</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.ml.inference.utils</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="c1"># pytype: skip-file</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd">Util/helper functions used in apache_beam.ml.inference.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="kn">import</span> <span class="nn">os</span>
<span class="kn">from</span> <span class="nn">functools</span> <span class="kn">import</span> <span class="n">partial</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Dict</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Iterable</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Optional</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Union</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.io.fileio</span> <span class="kn">import</span> <span class="n">EmptyMatchTreatment</span>
<span class="kn">from</span> <span class="nn">apache_beam.io.fileio</span> <span class="kn">import</span> <span class="n">MatchContinuously</span>
<span class="kn">from</span> <span class="nn">apache_beam.ml.inference.base</span> <span class="kn">import</span> <span class="n">ModelMetadata</span>
<span class="kn">from</span> <span class="nn">apache_beam.ml.inference.base</span> <span class="kn">import</span> <span class="n">PredictionResult</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">trigger</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">window</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.userstate</span> <span class="kn">import</span> <span class="n">CombiningValueStateSpec</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.timestamp</span> <span class="kn">import</span> <span class="n">MAX_TIMESTAMP</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils.timestamp</span> <span class="kn">import</span> <span class="n">Timestamp</span>
<span class="n">_START_TIME_STAMP</span> <span class="o">=</span> <span class="n">Timestamp</span><span class="o">.</span><span class="n">now</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_convert_to_result</span><span class="p">(</span>
<span class="n">batch</span><span class="p">:</span> <span class="n">Iterable</span><span class="p">,</span>
<span class="n">predictions</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Iterable</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">Iterable</span><span class="p">]],</span>
<span class="n">model_id</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterable</span><span class="p">[</span><span class="n">PredictionResult</span><span class="p">]:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">predictions</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="c1"># Go from one dictionary of type: {key_type1: Iterable&lt;val_type1&gt;,</span>
<span class="c1"># key_type2: Iterable&lt;val_type2&gt;, ...} where each Iterable is of</span>
<span class="c1"># length batch_size, to a list of dictionaries:</span>
<span class="c1"># [{key_type1: value_type1, key_type2: value_type2}]</span>
<span class="n">predictions_per_tensor</span> <span class="o">=</span> <span class="p">[</span>
<span class="nb">dict</span><span class="p">(</span><span class="nb">zip</span><span class="p">(</span><span class="n">predictions</span><span class="o">.</span><span class="n">keys</span><span class="p">(),</span> <span class="n">v</span><span class="p">))</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">predictions</span><span class="o">.</span><span class="n">values</span><span class="p">())</span>
<span class="p">]</span>
<span class="k">return</span> <span class="p">[</span>
<span class="n">PredictionResult</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">,</span> <span class="n">model_id</span><span class="p">)</span> <span class="k">for</span> <span class="n">x</span><span class="p">,</span>
<span class="n">y</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">batch</span><span class="p">,</span> <span class="n">predictions_per_tensor</span><span class="p">)</span>
<span class="p">]</span>
<span class="k">return</span> <span class="p">[</span><span class="n">PredictionResult</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">,</span> <span class="n">model_id</span><span class="p">)</span> <span class="k">for</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">batch</span><span class="p">,</span> <span class="n">predictions</span><span class="p">)]</span>
<span class="k">class</span> <span class="nc">_ConvertIterToSingleton</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Internal only; No backwards compatibility.</span>
<span class="sd"> The MatchContinuously transform examines all files present in a given</span>
<span class="sd"> directory and returns those that have timestamps older than the</span>
<span class="sd"> pipeline&#39;s start time. This can produce an Iterable rather than a</span>
<span class="sd"> Singleton. This class only returns the file path when it is first</span>
<span class="sd"> encountered, and it is cached as part of the side input caching mechanism.</span>
<span class="sd"> If the path is seen again, it will not return anything.</span>
<span class="sd"> By doing this, we can ensure that the output of this transform can be wrapped</span>
<span class="sd"> with beam.pvalue.AsSingleton().</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">COUNT_STATE</span> <span class="o">=</span> <span class="n">CombiningValueStateSpec</span><span class="p">(</span><span class="s1">&#39;count&#39;</span><span class="p">,</span> <span class="n">combine_fn</span><span class="o">=</span><span class="nb">sum</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">count_state</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">StateParam</span><span class="p">(</span><span class="n">COUNT_STATE</span><span class="p">)):</span>
<span class="n">counter</span> <span class="o">=</span> <span class="n">count_state</span><span class="o">.</span><span class="n">read</span><span class="p">()</span>
<span class="k">if</span> <span class="n">counter</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">count_state</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">element</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span>
<span class="k">class</span> <span class="nc">_GetLatestFileByTimeStamp</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Internal only; No backwards compatibility.</span>
<span class="sd"> This DoFn checks the timestamps of files against the time that the pipeline</span>
<span class="sd"> began running. It returns the files that were modified after the pipeline</span>
<span class="sd"> started. If no such files are found, it returns a default file as fallback.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">TIME_STATE</span> <span class="o">=</span> <span class="n">CombiningValueStateSpec</span><span class="p">(</span>
<span class="s1">&#39;max&#39;</span><span class="p">,</span> <span class="n">combine_fn</span><span class="o">=</span><span class="n">partial</span><span class="p">(</span><span class="nb">max</span><span class="p">,</span> <span class="n">default</span><span class="o">=</span><span class="n">_START_TIME_STAMP</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element</span><span class="p">,</span> <span class="n">time_state</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">StateParam</span><span class="p">(</span><span class="n">TIME_STATE</span><span class="p">)):</span>
<span class="n">_</span><span class="p">,</span> <span class="n">file_metadata</span> <span class="o">=</span> <span class="n">element</span>
<span class="n">new_ts</span> <span class="o">=</span> <span class="n">file_metadata</span><span class="o">.</span><span class="n">last_updated_in_seconds</span>
<span class="n">old_ts</span> <span class="o">=</span> <span class="n">time_state</span><span class="o">.</span><span class="n">read</span><span class="p">()</span>
<span class="k">if</span> <span class="n">new_ts</span> <span class="o">&gt;</span> <span class="n">old_ts</span><span class="p">:</span>
<span class="n">time_state</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span>
<span class="n">time_state</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">new_ts</span><span class="p">)</span>
<span class="n">model_path</span> <span class="o">=</span> <span class="n">file_metadata</span><span class="o">.</span><span class="n">path</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">model_path</span> <span class="o">=</span> <span class="s1">&#39;&#39;</span>
<span class="n">model_name</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">splitext</span><span class="p">(</span><span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">basename</span><span class="p">(</span><span class="n">model_path</span><span class="p">))[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">return</span> <span class="p">[</span>
<span class="p">(</span><span class="n">model_path</span><span class="p">,</span> <span class="n">ModelMetadata</span><span class="p">(</span><span class="n">model_id</span><span class="o">=</span><span class="n">model_path</span><span class="p">,</span> <span class="n">model_name</span><span class="o">=</span><span class="n">model_name</span><span class="p">))</span>
<span class="p">]</span>
<div class="viewcode-block" id="WatchFilePattern"><a class="viewcode-back" href="../../../../apache_beam.ml.inference.utils.html#apache_beam.ml.inference.utils.WatchFilePattern">[docs]</a><span class="k">class</span> <span class="nc">WatchFilePattern</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">file_pattern</span><span class="p">,</span>
<span class="n">interval</span><span class="o">=</span><span class="mi">360</span><span class="p">,</span>
<span class="n">stop_timestamp</span><span class="o">=</span><span class="n">MAX_TIMESTAMP</span><span class="p">,</span>
<span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Watches a directory for updates to files matching a given file pattern.</span>
<span class="sd"> Args:</span>
<span class="sd"> file_pattern: The file path to read from as a local file path or a</span>
<span class="sd"> GCS ``gs://`` path. The path can contain glob characters</span>
<span class="sd"> (``*``, ``?``, and ``[...]`` sets).</span>
<span class="sd"> interval: Interval at which to check for files matching file_pattern</span>
<span class="sd"> in seconds.</span>
<span class="sd"> stop_timestamp: Timestamp after which no more files will be checked.</span>
<span class="sd"> **Note**:</span>
<span class="sd"> 1. Any previously used filenames cannot be reused. If a file is added</span>
<span class="sd"> or updated to a previously used filename, this transform will ignore</span>
<span class="sd"> that update. To trigger a model update, always upload a file with</span>
<span class="sd"> unique name.</span>
<span class="sd"> 2. Initially, before the pipeline startup time, WatchFilePattern expects</span>
<span class="sd"> at least one file present that matches the file_pattern.</span>
<span class="sd"> 3. This transform is supported in streaming mode since</span>
<span class="sd"> MatchContinuously produces an unbounded source. Running in batch</span>
<span class="sd"> mode can lead to undesired results or result in pipeline being stuck.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">file_pattern</span> <span class="o">=</span> <span class="n">file_pattern</span>
<span class="bp">self</span><span class="o">.</span><span class="n">interval</span> <span class="o">=</span> <span class="n">interval</span>
<span class="bp">self</span><span class="o">.</span><span class="n">stop_timestamp</span> <span class="o">=</span> <span class="n">stop_timestamp</span>
<div class="viewcode-block" id="WatchFilePattern.expand"><a class="viewcode-back" href="../../../../apache_beam.ml.inference.utils.html#apache_beam.ml.inference.utils.WatchFilePattern.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">PCollection</span><span class="p">[</span><span class="n">ModelMetadata</span><span class="p">]:</span>
<span class="k">return</span> <span class="p">(</span>
<span class="n">pcoll</span>
<span class="o">|</span> <span class="s1">&#39;MatchContinuously&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">MatchContinuously</span><span class="p">(</span>
<span class="n">file_pattern</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">file_pattern</span><span class="p">,</span>
<span class="n">interval</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">interval</span><span class="p">,</span>
<span class="n">stop_timestamp</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">stop_timestamp</span><span class="p">,</span>
<span class="n">empty_match_treatment</span><span class="o">=</span><span class="n">EmptyMatchTreatment</span><span class="o">.</span><span class="n">DISALLOW</span><span class="p">)</span>
<span class="o">|</span> <span class="s2">&quot;AttachKey&quot;</span> <span class="o">&gt;&gt;</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="n">x</span><span class="o">.</span><span class="n">path</span><span class="p">,</span> <span class="n">x</span><span class="p">))</span>
<span class="o">|</span> <span class="s2">&quot;GetLatestFileMetaData&quot;</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="n">_GetLatestFileByTimeStamp</span><span class="p">())</span>
<span class="o">|</span> <span class="s2">&quot;AcceptNewSideInputOnly&quot;</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="n">_ConvertIterToSingleton</span><span class="p">())</span>
<span class="o">|</span> <span class="s1">&#39;ApplyGlobalWindow&#39;</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">transforms</span><span class="o">.</span><span class="n">WindowInto</span><span class="p">(</span>
<span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="p">(),</span>
<span class="n">trigger</span><span class="o">=</span><span class="n">trigger</span><span class="o">.</span><span class="n">Repeatedly</span><span class="p">(</span><span class="n">trigger</span><span class="o">.</span><span class="n">AfterProcessingTime</span><span class="p">(</span><span class="mi">1</span><span class="p">)),</span>
<span class="n">accumulation_mode</span><span class="o">=</span><span class="n">trigger</span><span class="o">.</span><span class="n">AccumulationMode</span><span class="o">.</span><span class="n">DISCARDING</span><span class="p">))</span></div></div>
</pre></div>
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/rtfd/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.Navigation.enable(true);
});
</script>
</body>
</html>