| |
| |
| <!DOCTYPE html> |
| <!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]--> |
| <!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]--> |
| <head> |
| <meta charset="utf-8"> |
| |
| <meta name="viewport" content="width=device-width, initial-scale=1.0"> |
| |
| <title>apache_beam.transforms.util — 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> »</li> |
| |
| <li><a href="../../index.html">Module code</a> »</li> |
| |
| <li>apache_beam.transforms.util</li> |
| |
| |
| <li class="wy-breadcrumbs-aside"> |
| |
| </li> |
| |
| </ul> |
| |
| |
| <hr/> |
| </div> |
| <div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article"> |
| <div itemprop="articleBody"> |
| |
| <h1>Source code for apache_beam.transforms.util</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">"""Simple utility PTransforms.</span> |
| <span class="sd">"""</span> |
| |
| <span class="c1"># pytype: skip-file</span> |
| |
| <span class="kn">import</span> <span class="nn">collections</span> |
| <span class="kn">import</span> <span class="nn">contextlib</span> |
| <span class="kn">import</span> <span class="nn">logging</span> |
| <span class="kn">import</span> <span class="nn">random</span> |
| <span class="kn">import</span> <span class="nn">re</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">uuid</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TYPE_CHECKING</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Iterable</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">List</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Tuple</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TypeVar</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Union</span> |
| |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">coders</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">typehints</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.metrics</span> <span class="kn">import</span> <span class="n">Metrics</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability</span> <span class="kn">import</span> <span class="n">common_urns</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">beam_runner_api_pb2</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pvalue</span> <span class="kn">import</span> <span class="n">AsSideInput</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.combiners</span> <span class="kn">import</span> <span class="n">CountCombineFn</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">CombinePerKey</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">Create</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">DoFn</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">FlatMap</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">Flatten</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">GroupByKey</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">Map</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">MapTuple</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">ParDo</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">Windowing</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="kn">import</span> <span class="n">PTransform</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.ptransform</span> <span class="kn">import</span> <span class="n">ptransform_fn</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.timeutil</span> <span class="kn">import</span> <span class="n">TimeDomain</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">AccumulationMode</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.trigger</span> <span class="kn">import</span> <span class="n">Always</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.userstate</span> <span class="kn">import</span> <span class="n">BagStateSpec</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.transforms.userstate</span> <span class="kn">import</span> <span class="n">TimerSpec</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.userstate</span> <span class="kn">import</span> <span class="n">on_timer</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">NonMergingWindowFn</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">TimestampCombiner</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.window</span> <span class="kn">import</span> <span class="n">TimestampedValue</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.typehints</span> <span class="kn">import</span> <span class="n">trivial_inference</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.typehints.decorators</span> <span class="kn">import</span> <span class="n">get_signature</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.typehints.sharded_key_type</span> <span class="kn">import</span> <span class="n">ShardedKeyType</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="kn">import</span> <span class="n">windowed_value</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.utils.annotations</span> <span class="kn">import</span> <span class="n">deprecated</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.utils.annotations</span> <span class="kn">import</span> <span class="n">experimental</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.utils.sharded_key</span> <span class="kn">import</span> <span class="n">ShardedKey</span> |
| |
| <span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">pvalue</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.pipeline_context</span> <span class="kn">import</span> <span class="n">PipelineContext</span> |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'BatchElements'</span><span class="p">,</span> |
| <span class="s1">'CoGroupByKey'</span><span class="p">,</span> |
| <span class="s1">'Distinct'</span><span class="p">,</span> |
| <span class="s1">'Keys'</span><span class="p">,</span> |
| <span class="s1">'KvSwap'</span><span class="p">,</span> |
| <span class="s1">'LogElements'</span><span class="p">,</span> |
| <span class="s1">'Regex'</span><span class="p">,</span> |
| <span class="s1">'Reify'</span><span class="p">,</span> |
| <span class="s1">'RemoveDuplicates'</span><span class="p">,</span> |
| <span class="s1">'Reshuffle'</span><span class="p">,</span> |
| <span class="s1">'ToString'</span><span class="p">,</span> |
| <span class="s1">'Values'</span><span class="p">,</span> |
| <span class="s1">'WithKeys'</span><span class="p">,</span> |
| <span class="s1">'GroupIntoBatches'</span> |
| <span class="p">]</span> |
| |
| <span class="n">K</span> <span class="o">=</span> <span class="n">TypeVar</span><span class="p">(</span><span class="s1">'K'</span><span class="p">)</span> |
| <span class="n">V</span> <span class="o">=</span> <span class="n">TypeVar</span><span class="p">(</span><span class="s1">'V'</span><span class="p">)</span> |
| <span class="n">T</span> <span class="o">=</span> <span class="n">TypeVar</span><span class="p">(</span><span class="s1">'T'</span><span class="p">)</span> |
| |
| |
| <div class="viewcode-block" id="CoGroupByKey"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.CoGroupByKey">[docs]</a><span class="k">class</span> <span class="nc">CoGroupByKey</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Groups results across several PCollections by key.</span> |
| |
| <span class="sd"> Given an input dict of serializable keys (called "tags") to 0 or more</span> |
| <span class="sd"> PCollections of (key, value) tuples, it creates a single output PCollection</span> |
| <span class="sd"> of (key, value) tuples whose keys are the unique input keys from all inputs,</span> |
| <span class="sd"> and whose values are dicts mapping each tag to an iterable of whatever values</span> |
| <span class="sd"> were under the key in the corresponding PCollection, in this manner::</span> |
| |
| <span class="sd"> ('some key', {'tag1': ['value 1 under "some key" in pcoll1',</span> |
| <span class="sd"> 'value 2 under "some key" in pcoll1',</span> |
| <span class="sd"> ...],</span> |
| <span class="sd"> 'tag2': ... ,</span> |
| <span class="sd"> ... })</span> |
| |
| <span class="sd"> where `[]` refers to an iterable, not a list.</span> |
| |
| <span class="sd"> For example, given::</span> |
| |
| <span class="sd"> {'tag1': pc1, 'tag2': pc2, 333: pc3}</span> |
| |
| <span class="sd"> where::</span> |
| |
| <span class="sd"> pc1 = beam.Create([(k1, v1)]))</span> |
| <span class="sd"> pc2 = beam.Create([])</span> |
| <span class="sd"> pc3 = beam.Create([(k1, v31), (k1, v32), (k2, v33)])</span> |
| |
| <span class="sd"> The output PCollection would consist of items::</span> |
| |
| <span class="sd"> [(k1, {'tag1': [v1], 'tag2': [], 333: [v31, v32]}),</span> |
| <span class="sd"> (k2, {'tag1': [], 'tag2': [], 333: [v33]})]</span> |
| |
| <span class="sd"> where `[]` refers to an iterable, not a list.</span> |
| |
| <span class="sd"> CoGroupByKey also works for tuples, lists, or other flat iterables of</span> |
| <span class="sd"> PCollections, in which case the values of the resulting PCollections</span> |
| <span class="sd"> will be tuples whose nth value is the iterable of values from the nth</span> |
| <span class="sd"> PCollection---conceptually, the "tags" are the indices into the input.</span> |
| <span class="sd"> Thus, for this input::</span> |
| |
| <span class="sd"> (pc1, pc2, pc3)</span> |
| |
| <span class="sd"> the output would be::</span> |
| |
| <span class="sd"> [(k1, ([v1], [], [v31, v32]),</span> |
| <span class="sd"> (k2, ([], [], [v33]))]</span> |
| |
| <span class="sd"> where, again, `[]` refers to an iterable, not a list.</span> |
| |
| <span class="sd"> Attributes:</span> |
| <span class="sd"> **kwargs: Accepts a single named argument "pipeline", which specifies the</span> |
| <span class="sd"> pipeline that "owns" this PTransform. Ordinarily CoGroupByKey can obtain</span> |
| <span class="sd"> this information from one of the input PCollections, but if there are none</span> |
| <span class="sd"> (or if there's a chance there may be none), this argument is the only way</span> |
| <span class="sd"> to provide pipeline information, and should be considered mandatory.</span> |
| <span class="sd"> """</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="o">*</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">=</span> <span class="n">pipeline</span> |
| |
| <span class="k">def</span> <span class="nf">_extract_input_pvalues</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalueish</span><span class="p">):</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="c1"># If this works, it's a dict.</span> |
| <span class="k">return</span> <span class="n">pvalueish</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">pvalueish</span><span class="o">.</span><span class="n">values</span><span class="p">())</span> |
| <span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span> |
| <span class="c1"># Cast iterables a tuple so we can do re-iteration.</span> |
| <span class="n">pcolls</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">pvalueish</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">pcolls</span><span class="p">,</span> <span class="n">pcolls</span> |
| |
| <div class="viewcode-block" id="CoGroupByKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.CoGroupByKey.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">pcolls</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">pcolls</span><span class="p">:</span> |
| <span class="n">pcolls</span> <span class="o">=</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">Create</span><span class="p">([]),</span> <span class="p">)</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pcolls</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span> |
| <span class="n">tags</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">pcolls</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| <span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">tag</span><span class="p">)</span> <span class="o"><</span> <span class="mi">10</span> <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="n">tags</span><span class="p">):</span> |
| <span class="c1"># Small, string tags. Pass them as data.</span> |
| <span class="n">pcolls_dict</span> <span class="o">=</span> <span class="n">pcolls</span> |
| <span class="n">restore_tags</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Pass the tags in the restore_tags closure.</span> |
| <span class="n">tags</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">pcolls</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| <span class="n">pcolls_dict</span> <span class="o">=</span> <span class="p">{</span><span class="nb">str</span><span class="p">(</span><span class="n">ix</span><span class="p">):</span> <span class="n">pcolls</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span> <span class="k">for</span> <span class="p">(</span><span class="n">ix</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">tags</span><span class="p">)}</span> |
| <span class="n">restore_tags</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">vs</span><span class="p">:</span> <span class="p">{</span> |
| <span class="n">tag</span><span class="p">:</span> <span class="n">vs</span><span class="p">[</span><span class="nb">str</span><span class="p">(</span><span class="n">ix</span><span class="p">)]</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">ix</span><span class="p">,</span> <span class="n">tag</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">tags</span><span class="p">)</span> |
| <span class="p">}</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Tags are tuple indices.</span> |
| <span class="n">tags</span> <span class="o">=</span> <span class="p">[</span><span class="nb">str</span><span class="p">(</span><span class="n">ix</span><span class="p">)</span> <span class="k">for</span> <span class="n">ix</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">pcolls</span><span class="p">))]</span> |
| <span class="n">pcolls_dict</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">(</span><span class="nb">zip</span><span class="p">(</span><span class="n">tags</span><span class="p">,</span> <span class="n">pcolls</span><span class="p">))</span> |
| <span class="n">restore_tags</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">vs</span><span class="p">:</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">vs</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span> <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="n">tags</span><span class="p">)</span> |
| |
| <span class="n">input_key_types</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">input_value_types</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls_dict</span><span class="o">.</span><span class="n">values</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">typehints</span><span class="o">.</span><span class="n">trivial_inference</span><span class="o">.</span><span class="n">key_value_types</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">input_key_types</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">key_type</span><span class="p">)</span> |
| <span class="n">input_value_types</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value_type</span><span class="p">)</span> |
| <span class="n">output_key_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Union</span><span class="p">[</span><span class="nb">tuple</span><span class="p">(</span><span class="n">input_key_types</span><span class="p">)]</span> |
| <span class="n">iterable_input_value_types</span> <span class="o">=</span> <span class="nb">tuple</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">t</span><span class="p">]</span> <span class="k">for</span> <span class="n">t</span> <span class="ow">in</span> <span class="n">input_value_types</span><span class="p">)</span> |
| |
| <span class="n">output_value_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Dict</span><span class="p">[</span> |
| <span class="nb">str</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Union</span><span class="p">[</span><span class="n">iterable_input_value_types</span> <span class="ow">or</span> <span class="p">[</span><span class="n">typehints</span><span class="o">.</span><span class="n">Any</span><span class="p">]]]</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">pcolls_dict</span> |
| <span class="o">|</span> <span class="s1">'CoGroupByKeyImpl'</span> <span class="o">>></span> |
| <span class="n">_CoGBKImpl</span><span class="p">(</span><span class="n">pipeline</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span> |
| <span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">output_key_type</span><span class="p">,</span> <span class="n">output_value_type</span><span class="p">]))</span> |
| |
| <span class="k">if</span> <span class="n">restore_tags</span><span class="p">:</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pcolls</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span> |
| <span class="n">dict_key_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Union</span><span class="p">[</span><span class="nb">tuple</span><span class="p">(</span> |
| <span class="n">trivial_inference</span><span class="o">.</span><span class="n">instance_to_type</span><span class="p">(</span><span class="n">tag</span><span class="p">)</span> <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="n">tags</span><span class="p">)]</span> |
| <span class="n">output_value_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Dict</span><span class="p">[</span> |
| <span class="n">dict_key_type</span><span class="p">,</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Union</span><span class="p">[</span><span class="n">iterable_input_value_types</span><span class="p">]]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">output_value_type</span> <span class="o">=</span> <span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">iterable_input_value_types</span><span class="p">]</span> |
| <span class="n">result</span> <span class="o">|=</span> <span class="s1">'RestoreTags'</span> <span class="o">>></span> <span class="n">MapTuple</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">k</span><span class="p">,</span> <span class="n">vs</span><span class="p">:</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">restore_tags</span><span class="p">(</span><span class="n">vs</span><span class="p">)))</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span> |
| <span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span><span class="n">output_key_type</span><span class="p">,</span> <span class="n">output_value_type</span><span class="p">])</span> |
| |
| <span class="k">return</span> <span class="n">result</span></div></div> |
| |
| |
| <span class="k">class</span> <span class="nc">_CoGBKImpl</span><span class="p">(</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="o">*</span><span class="p">,</span> <span class="n">pipeline</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">=</span> <span class="n">pipeline</span> |
| |
| <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">pcolls</span><span class="p">):</span> |
| <span class="c1"># Check input PCollections for PCollection-ness, and that they all belong</span> |
| <span class="c1"># to the same pipeline.</span> |
| <span class="k">for</span> <span class="n">pcoll</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="o">.</span><span class="n">values</span><span class="p">():</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_check_pcollection</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">:</span> |
| <span class="k">assert</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">,</span> <span class="p">(</span> |
| <span class="s1">'All input PCollections must belong to the same pipeline.'</span><span class="p">)</span> |
| |
| <span class="n">tags</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">pcolls</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| |
| <span class="k">def</span> <span class="nf">add_tag</span><span class="p">(</span><span class="n">tag</span><span class="p">):</span> |
| <span class="k">return</span> <span class="k">lambda</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">v</span><span class="p">))</span> |
| |
| <span class="k">def</span> <span class="nf">collect_values</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">tagged_values</span><span class="p">):</span> |
| <span class="n">grouped_values</span> <span class="o">=</span> <span class="p">{</span><span class="n">tag</span><span class="p">:</span> <span class="p">[]</span> <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="n">tags</span><span class="p">}</span> |
| <span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">tagged_values</span><span class="p">:</span> |
| <span class="n">grouped_values</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">key</span><span class="p">,</span> <span class="n">grouped_values</span> |
| |
| <span class="k">return</span> <span class="p">([</span> |
| <span class="n">pcoll</span> |
| <span class="o">|</span> <span class="s1">'Tag[</span><span class="si">%s</span><span class="s1">]'</span> <span class="o">%</span> <span class="n">tag</span> <span class="o">>></span> <span class="n">MapTuple</span><span class="p">(</span><span class="n">add_tag</span><span class="p">(</span><span class="n">tag</span><span class="p">))</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">)</span> <span class="ow">in</span> <span class="n">pcolls</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="p">]</span> |
| <span class="o">|</span> <span class="n">Flatten</span><span class="p">(</span><span class="n">pipeline</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span> |
| <span class="o">|</span> <span class="n">GroupByKey</span><span class="p">()</span> |
| <span class="o">|</span> <span class="n">MapTuple</span><span class="p">(</span><span class="n">collect_values</span><span class="p">))</span> |
| |
| |
| <div class="viewcode-block" id="Keys"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Keys">[docs]</a><span class="nd">@ptransform_fn</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">K</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">Keys</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">'Keys'</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span> |
| <span class="w"> </span><span class="sd">"""Produces a PCollection of first elements of 2-tuples in a PCollection."""</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">label</span> <span class="o">>></span> <span class="n">MapTuple</span><span class="p">(</span><span class="k">lambda</span> <span class="n">k</span><span class="p">,</span> <span class="n">_</span><span class="p">:</span> <span class="n">k</span><span class="p">)</span></div> |
| |
| |
| <div class="viewcode-block" id="Values"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Values">[docs]</a><span class="nd">@ptransform_fn</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">V</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">Values</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">'Values'</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span> |
| <span class="w"> </span><span class="sd">"""Produces a PCollection of second elements of 2-tuples in a PCollection."""</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">label</span> <span class="o">>></span> <span class="n">MapTuple</span><span class="p">(</span><span class="k">lambda</span> <span class="n">_</span><span class="p">,</span> <span class="n">v</span><span class="p">:</span> <span class="n">v</span><span class="p">)</span></div> |
| |
| |
| <div class="viewcode-block" id="KvSwap"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.KvSwap">[docs]</a><span class="nd">@ptransform_fn</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">V</span><span class="p">,</span> <span class="n">K</span><span class="p">])</span> |
| <span class="k">def</span> <span class="nf">KvSwap</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">'KvSwap'</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span> |
| <span class="w"> </span><span class="sd">"""Produces a PCollection reversing 2-tuples in a PCollection."""</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">label</span> <span class="o">>></span> <span class="n">MapTuple</span><span class="p">(</span><span class="k">lambda</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">k</span><span class="p">))</span></div> |
| |
| |
| <div class="viewcode-block" id="Distinct"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Distinct">[docs]</a><span class="nd">@ptransform_fn</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">Distinct</span><span class="p">(</span><span class="n">pcoll</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span> |
| <span class="w"> </span><span class="sd">"""Produces a PCollection containing distinct elements of a PCollection."""</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">pcoll</span> |
| <span class="o">|</span> <span class="s1">'ToPairs'</span> <span class="o">>></span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span> |
| <span class="o">|</span> <span class="s1">'Group'</span> <span class="o">>></span> <span class="n">CombinePerKey</span><span class="p">(</span><span class="k">lambda</span> <span class="n">vs</span><span class="p">:</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="o">|</span> <span class="s1">'Distinct'</span> <span class="o">>></span> <span class="n">Keys</span><span class="p">())</span></div> |
| |
| |
| <div class="viewcode-block" id="RemoveDuplicates"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.RemoveDuplicates">[docs]</a><span class="nd">@deprecated</span><span class="p">(</span><span class="n">since</span><span class="o">=</span><span class="s1">'2.12'</span><span class="p">,</span> <span class="n">current</span><span class="o">=</span><span class="s1">'Distinct'</span><span class="p">)</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">RemoveDuplicates</span><span class="p">(</span><span class="n">pcoll</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Produces a PCollection containing distinct elements of a PCollection."""</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">'RemoveDuplicates'</span> <span class="o">>></span> <span class="n">Distinct</span><span class="p">()</span></div> |
| |
| |
| <span class="k">class</span> <span class="nc">_BatchSizeEstimator</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Estimates the best size for batches given historical timing.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">_MAX_DATA_POINTS</span> <span class="o">=</span> <span class="mi">100</span> |
| <span class="n">_MAX_GROWTH_FACTOR</span> <span class="o">=</span> <span class="mi">2</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">min_batch_size</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span> |
| <span class="n">max_batch_size</span><span class="o">=</span><span class="mi">10000</span><span class="p">,</span> |
| <span class="n">target_batch_overhead</span><span class="o">=</span><span class="mf">.05</span><span class="p">,</span> |
| <span class="n">target_batch_duration_secs</span><span class="o">=</span><span class="mi">10</span><span class="p">,</span> |
| <span class="n">target_batch_duration_secs_including_fixed_cost</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">variance</span><span class="o">=</span><span class="mf">0.25</span><span class="p">,</span> |
| <span class="n">clock</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">ignore_first_n_seen_per_batch_size</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">record_metrics</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">min_batch_size</span> <span class="o">></span> <span class="n">max_batch_size</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"Minimum (</span><span class="si">%s</span><span class="s2">) must not be greater than maximum (</span><span class="si">%s</span><span class="s2">)"</span> <span class="o">%</span> |
| <span class="p">(</span><span class="n">min_batch_size</span><span class="p">,</span> <span class="n">max_batch_size</span><span class="p">))</span> |
| <span class="k">if</span> <span class="n">target_batch_overhead</span> <span class="ow">and</span> <span class="ow">not</span> <span class="mi">0</span> <span class="o"><</span> <span class="n">target_batch_overhead</span> <span class="o"><=</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"target_batch_overhead (</span><span class="si">%s</span><span class="s2">) must be between 0 and 1"</span> <span class="o">%</span> |
| <span class="p">(</span><span class="n">target_batch_overhead</span><span class="p">))</span> |
| <span class="k">if</span> <span class="n">target_batch_duration_secs</span> <span class="ow">and</span> <span class="n">target_batch_duration_secs</span> <span class="o"><=</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"target_batch_duration_secs (</span><span class="si">%s</span><span class="s2">) must be positive"</span> <span class="o">%</span> |
| <span class="p">(</span><span class="n">target_batch_duration_secs</span><span class="p">))</span> |
| <span class="k">if</span> <span class="p">(</span><span class="n">target_batch_duration_secs_including_fixed_cost</span> <span class="ow">and</span> |
| <span class="n">target_batch_duration_secs_including_fixed_cost</span> <span class="o"><=</span> <span class="mi">0</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"target_batch_duration_secs_including_fixed_cost "</span> |
| <span class="s2">"(</span><span class="si">%s</span><span class="s2">) must be positive"</span> <span class="o">%</span> |
| <span class="p">(</span><span class="n">target_batch_duration_secs_including_fixed_cost</span><span class="p">))</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="p">(</span><span class="n">target_batch_overhead</span> <span class="ow">or</span> <span class="n">target_batch_duration_secs</span> <span class="ow">or</span> |
| <span class="n">target_batch_duration_secs_including_fixed_cost</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"At least one of target_batch_overhead or "</span> |
| <span class="s2">"target_batch_duration_secs or "</span> |
| <span class="s2">"target_batch_duration_secs_including_fixed_cost must be positive."</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">ignore_first_n_seen_per_batch_size</span> <span class="o"><</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'ignore_first_n_seen_per_batch_size (</span><span class="si">%s</span><span class="s1">) must be non '</span> |
| <span class="s1">'negative'</span> <span class="o">%</span> <span class="p">(</span><span class="n">ignore_first_n_seen_per_batch_size</span><span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">=</span> <span class="n">min_batch_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span> <span class="o">=</span> <span class="n">max_batch_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_overhead</span> <span class="o">=</span> <span class="n">target_batch_overhead</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs</span> <span class="o">=</span> <span class="n">target_batch_duration_secs</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs_including_fixed_cost</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">target_batch_duration_secs_including_fixed_cost</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_variance</span> <span class="o">=</span> <span class="n">variance</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_clock</span> <span class="o">=</span> <span class="n">clock</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_data</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ignore_next_timing</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ignore_first_n_seen_per_batch_size</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">ignore_first_n_seen_per_batch_size</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_num_seen</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_replay_last_batch_size</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_record_metrics</span> <span class="o">=</span> <span class="n">record_metrics</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_element_count</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_count</span> <span class="o">=</span> <span class="mi">0</span> |
| |
| <span class="k">if</span> <span class="n">record_metrics</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_size_distribution</span> <span class="o">=</span> <span class="n">Metrics</span><span class="o">.</span><span class="n">distribution</span><span class="p">(</span> |
| <span class="s1">'BatchElements'</span><span class="p">,</span> <span class="s1">'batch_size'</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_time_distribution</span> <span class="o">=</span> <span class="n">Metrics</span><span class="o">.</span><span class="n">distribution</span><span class="p">(</span> |
| <span class="s1">'BatchElements'</span><span class="p">,</span> <span class="s1">'msec_per_batch'</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_size_distribution</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_time_distribution</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="c1"># Beam distributions only accept integer values, so we use this to</span> |
| <span class="c1"># accumulate under-reported values until they add up to whole milliseconds.</span> |
| <span class="c1"># (Milliseconds are chosen because that's conventionally used elsewhere in</span> |
| <span class="c1"># profiling-style counters.)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_remainder_msecs</span> <span class="o">=</span> <span class="mi">0</span> |
| |
| <span class="k">def</span> <span class="nf">ignore_next_timing</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Call to indicate the next timing should be ignored.</span> |
| |
| <span class="sd"> For example, the first emit of a ParDo operation is known to be anomalous</span> |
| <span class="sd"> due to setup that may occur.</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ignore_next_timing</span> <span class="o">=</span> <span class="kc">True</span> |
| |
| <span class="nd">@contextlib</span><span class="o">.</span><span class="n">contextmanager</span> |
| <span class="k">def</span> <span class="nf">record_time</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">batch_size</span><span class="p">):</span> |
| <span class="n">start</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_clock</span><span class="p">()</span> |
| <span class="k">yield</span> |
| <span class="n">elapsed</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_clock</span><span class="p">()</span> <span class="o">-</span> <span class="n">start</span> |
| <span class="n">elapsed_msec</span> <span class="o">=</span> <span class="mf">1e3</span> <span class="o">*</span> <span class="n">elapsed</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_remainder_msecs</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_record_metrics</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_size_distribution</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">batch_size</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_time_distribution</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="nb">int</span><span class="p">(</span><span class="n">elapsed_msec</span><span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_element_count</span> <span class="o">+=</span> <span class="n">batch_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_count</span> <span class="o">+=</span> <span class="mi">1</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_remainder_msecs</span> <span class="o">=</span> <span class="n">elapsed_msec</span> <span class="o">-</span> <span class="nb">int</span><span class="p">(</span><span class="n">elapsed_msec</span><span class="p">)</span> |
| <span class="c1"># If we ignore the next timing, replay the batch size to get accurate</span> |
| <span class="c1"># timing.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ignore_next_timing</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ignore_next_timing</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_replay_last_batch_size</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">batch_size</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">batch_size</span><span class="p">,</span> <span class="n">elapsed</span><span class="p">))</span> |
| <span class="k">if</span> <span class="nb">len</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="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_MAX_DATA_POINTS</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_thin_data</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="nf">_thin_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># Make sure we don't change the parity of len(self._data)</span> |
| <span class="c1"># As it's used below to alternate jitter.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">randrange</span><span class="p">(</span><span class="nb">len</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="o">//</span> <span class="mi">4</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">pop</span><span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">randrange</span><span class="p">(</span><span class="nb">len</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="o">//</span> <span class="mi">2</span><span class="p">))</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">linear_regression_no_numpy</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">):</span> |
| <span class="c1"># Least squares fit for y = a + bx over all points.</span> |
| <span class="n">n</span> <span class="o">=</span> <span class="nb">float</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">xs</span><span class="p">))</span> |
| <span class="n">xbar</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">xs</span><span class="p">)</span> <span class="o">/</span> <span class="n">n</span> |
| <span class="n">ybar</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">ys</span><span class="p">)</span> <span class="o">/</span> <span class="n">n</span> |
| <span class="k">if</span> <span class="n">xbar</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">ybar</span><span class="p">,</span> <span class="mi">0</span> |
| <span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="n">xs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">==</span> <span class="n">x</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">xs</span><span class="p">):</span> |
| <span class="c1"># Simply use the mean if all values in xs are same.</span> |
| <span class="k">return</span> <span class="mi">0</span><span class="p">,</span> <span class="n">ybar</span> <span class="o">/</span> <span class="n">xbar</span> |
| <span class="n">b</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">sum</span><span class="p">([(</span><span class="n">x</span> <span class="o">-</span> <span class="n">xbar</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">y</span> <span class="o">-</span> <span class="n">ybar</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">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">)])</span> <span class="o">/</span> <span class="nb">sum</span><span class="p">([(</span><span class="n">x</span> <span class="o">-</span> <span class="n">xbar</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">xs</span><span class="p">]))</span> |
| <span class="n">a</span> <span class="o">=</span> <span class="n">ybar</span> <span class="o">-</span> <span class="n">b</span> <span class="o">*</span> <span class="n">xbar</span> |
| <span class="k">return</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">linear_regression_numpy</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">):</span> |
| <span class="c1"># pylint: disable=wrong-import-order, wrong-import-position</span> |
| <span class="kn">import</span> <span class="nn">numpy</span> <span class="k">as</span> <span class="nn">np</span> |
| <span class="kn">from</span> <span class="nn">numpy</span> <span class="kn">import</span> <span class="nb">sum</span> |
| <span class="n">n</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">xs</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="n">xs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">==</span> <span class="n">x</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">xs</span><span class="p">):</span> |
| <span class="c1"># If all values of xs are same then fallback to linear_regression_no_numpy</span> |
| <span class="k">return</span> <span class="n">_BatchSizeEstimator</span><span class="o">.</span><span class="n">linear_regression_no_numpy</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">)</span> |
| <span class="n">xs</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">asarray</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="nb">float</span><span class="p">)</span> |
| <span class="n">ys</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">asarray</span><span class="p">(</span><span class="n">ys</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="nb">float</span><span class="p">)</span> |
| |
| <span class="c1"># First do a simple least squares fit for y = a + bx over all points.</span> |
| <span class="n">b</span><span class="p">,</span> <span class="n">a</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">polyfit</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">n</span> <span class="o"><</span> <span class="mi">10</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Refine this by throwing out outliers, according to Cook's distance.</span> |
| <span class="c1"># https://en.wikipedia.org/wiki/Cook%27s_distance</span> |
| <span class="n">sum_x</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">xs</span><span class="p">)</span> |
| <span class="n">sum_x2</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">xs</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> |
| <span class="n">errs</span> <span class="o">=</span> <span class="n">a</span> <span class="o">+</span> <span class="n">b</span> <span class="o">*</span> <span class="n">xs</span> <span class="o">-</span> <span class="n">ys</span> |
| <span class="n">s2</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">errs</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> <span class="o">/</span> <span class="p">(</span><span class="n">n</span> <span class="o">-</span> <span class="mi">2</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">s2</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="c1"># It's an exact fit!</span> |
| <span class="k">return</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span> |
| <span class="n">h</span> <span class="o">=</span> <span class="p">(</span><span class="n">sum_x2</span> <span class="o">-</span> <span class="mi">2</span> <span class="o">*</span> <span class="n">sum_x</span> <span class="o">*</span> <span class="n">xs</span> <span class="o">+</span> <span class="n">n</span> <span class="o">*</span> <span class="n">xs</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> <span class="o">/</span> <span class="p">(</span><span class="n">n</span> <span class="o">*</span> <span class="n">sum_x2</span> <span class="o">-</span> <span class="n">sum_x</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> |
| <span class="n">cook_ds</span> <span class="o">=</span> <span class="mf">0.5</span> <span class="o">/</span> <span class="n">s2</span> <span class="o">*</span> <span class="n">errs</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="p">(</span><span class="n">h</span> <span class="o">/</span> <span class="p">(</span><span class="mi">1</span> <span class="o">-</span> <span class="n">h</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> |
| |
| <span class="c1"># Re-compute the regression, excluding those points with Cook's distance</span> |
| <span class="c1"># greater than 0.5, and weighting by the inverse of x to give a more</span> |
| <span class="c1"># stable y-intercept (as small batches have relatively more information</span> |
| <span class="c1"># about the fixed overhead).</span> |
| <span class="n">weight</span> <span class="o">=</span> <span class="p">(</span><span class="n">cook_ds</span> <span class="o"><=</span> <span class="mf">0.5</span><span class="p">)</span> <span class="o">/</span> <span class="n">xs</span> |
| <span class="n">b</span><span class="p">,</span> <span class="n">a</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">polyfit</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">,</span> <span class="mi">1</span><span class="p">,</span> <span class="n">w</span><span class="o">=</span><span class="n">weight</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</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">import</span> <span class="nn">numpy</span> <span class="k">as</span> <span class="nn">np</span> |
| <span class="n">linear_regression</span> <span class="o">=</span> <span class="n">linear_regression_numpy</span> |
| <span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span> |
| <span class="n">linear_regression</span> <span class="o">=</span> <span class="n">linear_regression_no_numpy</span> |
| |
| <span class="k">def</span> <span class="nf">_calculate_next_batch_size</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> |
| <span class="k">elif</span> <span class="nb">len</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="o"><</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> |
| <span class="k">elif</span> <span class="nb">len</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="o"><</span> <span class="mi">2</span><span class="p">:</span> |
| <span class="c1"># Force some variety so we have distinct batch sizes on which to do</span> |
| <span class="c1"># linear regression below.</span> |
| <span class="k">return</span> <span class="nb">int</span><span class="p">(</span> |
| <span class="nb">max</span><span class="p">(</span> |
| <span class="nb">min</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">*</span> <span class="bp">self</span><span class="o">.</span><span class="n">_MAX_GROWTH_FACTOR</span><span class="p">),</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">+</span> <span class="mi">1</span><span class="p">))</span> |
| |
| <span class="c1"># There tends to be a lot of noise in the top quantile, which also</span> |
| <span class="c1"># has outsided influence in the regression. If we have enough data,</span> |
| <span class="c1"># Simply declare the top 20% to be outliers.</span> |
| <span class="n">trimmed_data</span> <span class="o">=</span> <span class="nb">sorted</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="nb">max</span><span class="p">(</span><span class="mi">20</span><span class="p">,</span> <span class="nb">len</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="o">*</span> <span class="mi">4</span> <span class="o">//</span> <span class="mi">5</span><span class="p">)]</span> |
| |
| <span class="c1"># Linear regression for y = a + bx, where x is batch size and y is time.</span> |
| <span class="n">xs</span><span class="p">,</span> <span class="n">ys</span> <span class="o">=</span> <span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">trimmed_data</span><span class="p">)</span> |
| <span class="n">a</span><span class="p">,</span> <span class="n">b</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">linear_regression</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">ys</span><span class="p">)</span> |
| |
| <span class="c1"># Avoid nonsensical or division-by-zero errors below due to noise.</span> |
| <span class="n">a</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="mf">1e-10</span><span class="p">)</span> |
| <span class="n">b</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="n">b</span><span class="p">,</span> <span class="mf">1e-20</span><span class="p">)</span> |
| |
| <span class="n">last_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_data</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">][</span><span class="mi">0</span><span class="p">]</span> |
| <span class="n">cap</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">last_batch_size</span> <span class="o">*</span> <span class="bp">self</span><span class="o">.</span><span class="n">_MAX_GROWTH_FACTOR</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span><span class="p">)</span> |
| |
| <span class="n">target</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_max_batch_size</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs_including_fixed_cost</span><span class="p">:</span> |
| <span class="c1"># Solution to</span> |
| <span class="c1"># a + b*x = self._target_batch_duration_secs_including_fixed_cost.</span> |
| <span class="n">target</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span> |
| <span class="n">target</span><span class="p">,</span> |
| <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs_including_fixed_cost</span> <span class="o">-</span> <span class="n">a</span><span class="p">)</span> <span class="o">/</span> <span class="n">b</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs</span><span class="p">:</span> |
| <span class="c1"># Solution to b*x = self._target_batch_duration_secs.</span> |
| <span class="c1"># We ignore the fixed cost in this computation as it has negligeabel</span> |
| <span class="c1"># impact when it is small and unhelpfully forces the minimum batch size</span> |
| <span class="c1"># when it is large.</span> |
| <span class="n">target</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">target</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_duration_secs</span> <span class="o">/</span> <span class="n">b</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_overhead</span><span class="p">:</span> |
| <span class="c1"># Solution to a / (a + b*x) = self._target_batch_overhead.</span> |
| <span class="n">target</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">target</span><span class="p">,</span> <span class="p">(</span><span class="n">a</span> <span class="o">/</span> <span class="n">b</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="mi">1</span> <span class="o">/</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_overhead</span> <span class="o">-</span> <span class="mi">1</span><span class="p">))</span> |
| |
| <span class="c1"># Avoid getting stuck at a single batch size (especially the minimal</span> |
| <span class="c1"># batch size) which would not allow us to extrapolate to other batch</span> |
| <span class="c1"># sizes.</span> |
| <span class="c1"># Jitter alternates between 0 and 1.</span> |
| <span class="n">jitter</span> <span class="o">=</span> <span class="nb">len</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="o">%</span> <span class="mi">2</span> |
| <span class="c1"># Smear our samples across a range centered at the target.</span> |
| <span class="k">if</span> <span class="nb">len</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="o">></span> <span class="mi">10</span><span class="p">:</span> |
| <span class="n">target</span> <span class="o">+=</span> <span class="nb">int</span><span class="p">(</span><span class="n">target</span> <span class="o">*</span> <span class="bp">self</span><span class="o">.</span><span class="n">_variance</span> <span class="o">*</span> <span class="mi">2</span> <span class="o">*</span> <span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">random</span><span class="p">()</span> <span class="o">-</span> <span class="mf">.5</span><span class="p">))</span> |
| |
| <span class="k">return</span> <span class="nb">int</span><span class="p">(</span><span class="nb">max</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_min_batch_size</span> <span class="o">+</span> <span class="n">jitter</span><span class="p">,</span> <span class="nb">min</span><span class="p">(</span><span class="n">target</span><span class="p">,</span> <span class="n">cap</span><span class="p">)))</span> |
| |
| <span class="k">def</span> <span class="nf">next_batch_size</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># Check if we should replay a previous batch size due to it not being</span> |
| <span class="c1"># recorded.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_replay_last_batch_size</span><span class="p">:</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_replay_last_batch_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_replay_last_batch_size</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_calculate_next_batch_size</span><span class="p">()</span> |
| |
| <span class="n">seen_count</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_num_seen</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">result</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span> <span class="o">+</span> <span class="mi">1</span> |
| <span class="k">if</span> <span class="n">seen_count</span> <span class="o"><=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ignore_first_n_seen_per_batch_size</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">ignore_next_timing</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_num_seen</span><span class="p">[</span><span class="n">result</span><span class="p">]</span> <span class="o">=</span> <span class="n">seen_count</span> |
| <span class="k">return</span> <span class="n">result</span> |
| |
| <span class="k">def</span> <span class="nf">stats</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="s2">"element_count=</span><span class="si">%s</span><span class="s2"> batch_count=</span><span class="si">%s</span><span class="s2"> next_batch_size=</span><span class="si">%s</span><span class="s2"> timings=</span><span class="si">%s</span><span class="s2">"</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_element_count</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_count</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_calculate_next_batch_size</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="k">class</span> <span class="nc">_GlobalWindowsBatchingDoFn</span><span class="p">(</span><span class="n">DoFn</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">batch_size_estimator</span><span class="p">,</span> <span class="n">element_size_fn</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span> <span class="o">=</span> <span class="n">batch_size_estimator</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_element_size_fn</span> <span class="o">=</span> <span class="n">element_size_fn</span> |
| |
| <span class="k">def</span> <span class="nf">start_bundle</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">_batch</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_running_batch_size</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span> |
| <span class="c1"># The first emit often involves non-trivial setup.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">ignore_next_timing</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="bp">self</span><span class="o">.</span><span class="n">_batch</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_running_batch_size</span> <span class="o">+=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_element_size_fn</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_running_batch_size</span> <span class="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span><span class="p">:</span> |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_running_batch_size</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="o">.</span><span class="n">windowed_value_at_end_of_window</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_running_batch_size</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch</span><span class="p">:</span> |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_running_batch_size</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">window</span><span class="o">.</span><span class="n">GlobalWindows</span><span class="o">.</span><span class="n">windowed_value_at_end_of_window</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batch</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_running_batch_size</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</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="s2">"BatchElements statistics: "</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">stats</span><span class="p">())</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_SizedBatch</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="bp">self</span><span class="o">.</span><span class="n">elements</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">size</span> <span class="o">=</span> <span class="mi">0</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_WindowAwareBatchingDoFn</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span> |
| |
| <span class="n">_MAX_LIVE_WINDOWS</span> <span class="o">=</span> <span class="mi">10</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">batch_size_estimator</span><span class="p">,</span> <span class="n">element_size_fn</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span> <span class="o">=</span> <span class="n">batch_size_estimator</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_element_size_fn</span> <span class="o">=</span> <span class="n">element_size_fn</span> |
| |
| <span class="k">def</span> <span class="nf">start_bundle</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">_batches</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="n">_SizedBatch</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span> |
| <span class="c1"># The first emit often involves non-trivial setup.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">ignore_next_timing</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">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">):</span> |
| <span class="n">batch</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">]</span> |
| <span class="n">batch</span><span class="o">.</span><span class="n">elements</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="n">batch</span><span class="o">.</span><span class="n">size</span> <span class="o">+=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_element_size_fn</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">batch</span><span class="o">.</span><span class="n">size</span> <span class="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span><span class="p">:</span> |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="n">batch</span><span class="o">.</span><span class="n">size</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span> |
| <span class="n">batch</span><span class="o">.</span><span class="n">elements</span><span class="p">,</span> <span class="n">window</span><span class="o">.</span><span class="n">max_timestamp</span><span class="p">(),</span> <span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="p">))</span> |
| <span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span> |
| <span class="k">elif</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">)</span> <span class="o">></span> <span class="bp">self</span><span class="o">.</span><span class="n">_MAX_LIVE_WINDOWS</span><span class="p">:</span> |
| <span class="n">window</span><span class="p">,</span> <span class="n">batch</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="o">.</span><span class="n">items</span><span class="p">(),</span> |
| <span class="n">key</span><span class="o">=</span><span class="k">lambda</span> <span class="n">window_batch</span><span class="p">:</span> <span class="n">window_batch</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span><span class="o">.</span><span class="n">size</span><span class="p">)</span> |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="n">batch</span><span class="o">.</span><span class="n">size</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span> |
| <span class="n">batch</span><span class="o">.</span><span class="n">elements</span><span class="p">,</span> <span class="n">window</span><span class="o">.</span><span class="n">max_timestamp</span><span class="p">(),</span> <span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="p">))</span> |
| <span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="p">[</span><span class="n">window</span><span class="p">]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">for</span> <span class="n">window</span><span class="p">,</span> <span class="n">batch</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">batch</span><span class="p">:</span> |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">record_time</span><span class="p">(</span><span class="n">batch</span><span class="o">.</span><span class="n">size</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span> |
| <span class="n">batch</span><span class="o">.</span><span class="n">elements</span><span class="p">,</span> <span class="n">window</span><span class="o">.</span><span class="n">max_timestamp</span><span class="p">(),</span> <span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batches</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_batch_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="o">.</span><span class="n">next_batch_size</span><span class="p">()</span> |
| |
| |
| <div class="viewcode-block" id="BatchElements"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.BatchElements">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">List</span><span class="p">[</span><span class="n">T</span><span class="p">])</span> |
| <span class="k">class</span> <span class="nc">BatchElements</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A Transform that batches elements for amortized processing.</span> |
| |
| <span class="sd"> This transform is designed to precede operations whose processing cost</span> |
| <span class="sd"> is of the form</span> |
| |
| <span class="sd"> time = fixed_cost + num_elements * per_element_cost</span> |
| |
| <span class="sd"> where the per element cost is (often significantly) smaller than the fixed</span> |
| <span class="sd"> cost and could be amortized over multiple elements. It consumes a PCollection</span> |
| <span class="sd"> of element type T and produces a PCollection of element type List[T].</span> |
| |
| <span class="sd"> This transform attempts to find the best batch size between the minimim</span> |
| <span class="sd"> and maximum parameters by profiling the time taken by (fused) downstream</span> |
| <span class="sd"> operations. For a fixed batch size, set the min and max to be equal.</span> |
| |
| <span class="sd"> Elements are batched per-window and batches emitted in the window</span> |
| <span class="sd"> corresponding to its contents.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> min_batch_size: (optional) the smallest size of a batch</span> |
| <span class="sd"> max_batch_size: (optional) the largest size of a batch</span> |
| <span class="sd"> target_batch_overhead: (optional) a target for fixed_cost / time,</span> |
| <span class="sd"> as used in the formula above</span> |
| <span class="sd"> target_batch_duration_secs: (optional) a target for total time per bundle,</span> |
| <span class="sd"> in seconds, excluding fixed cost</span> |
| <span class="sd"> target_batch_duration_secs_including_fixed_cost: (optional) a target for</span> |
| <span class="sd"> total time per bundle, in seconds, including fixed cost</span> |
| <span class="sd"> element_size_fn: (optional) A mapping of an element to its contribution to</span> |
| <span class="sd"> batch size, defaulting to every element having size 1. When provided,</span> |
| <span class="sd"> attempts to provide batches of optimal total size which may consist of</span> |
| <span class="sd"> a varying number of elements.</span> |
| <span class="sd"> variance: (optional) the permitted (relative) amount of deviation from the</span> |
| <span class="sd"> (estimated) ideal batch size used to produce a wider base for</span> |
| <span class="sd"> linear interpolation</span> |
| <span class="sd"> clock: (optional) an alternative to time.time for measuring the cost of</span> |
| <span class="sd"> donwstream operations (mostly for testing)</span> |
| <span class="sd"> record_metrics: (optional) whether or not to record beam metrics on</span> |
| <span class="sd"> distributions of the batch size. Defaults to True.</span> |
| <span class="sd"> """</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">min_batch_size</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span> |
| <span class="n">max_batch_size</span><span class="o">=</span><span class="mi">10000</span><span class="p">,</span> |
| <span class="n">target_batch_overhead</span><span class="o">=</span><span class="mf">.05</span><span class="p">,</span> |
| <span class="n">target_batch_duration_secs</span><span class="o">=</span><span class="mi">10</span><span class="p">,</span> |
| <span class="n">target_batch_duration_secs_including_fixed_cost</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="o">*</span><span class="p">,</span> |
| <span class="n">element_size_fn</span><span class="o">=</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="mi">1</span><span class="p">,</span> |
| <span class="n">variance</span><span class="o">=</span><span class="mf">0.25</span><span class="p">,</span> |
| <span class="n">clock</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">record_metrics</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span> <span class="o">=</span> <span class="n">_BatchSizeEstimator</span><span class="p">(</span> |
| <span class="n">min_batch_size</span><span class="o">=</span><span class="n">min_batch_size</span><span class="p">,</span> |
| <span class="n">max_batch_size</span><span class="o">=</span><span class="n">max_batch_size</span><span class="p">,</span> |
| <span class="n">target_batch_overhead</span><span class="o">=</span><span class="n">target_batch_overhead</span><span class="p">,</span> |
| <span class="n">target_batch_duration_secs</span><span class="o">=</span><span class="n">target_batch_duration_secs</span><span class="p">,</span> |
| <span class="n">target_batch_duration_secs_including_fixed_cost</span><span class="o">=</span><span class="p">(</span> |
| <span class="n">target_batch_duration_secs_including_fixed_cost</span><span class="p">),</span> |
| <span class="n">variance</span><span class="o">=</span><span class="n">variance</span><span class="p">,</span> |
| <span class="n">clock</span><span class="o">=</span><span class="n">clock</span><span class="p">,</span> |
| <span class="n">record_metrics</span><span class="o">=</span><span class="n">record_metrics</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_element_size_fn</span> <span class="o">=</span> <span class="n">element_size_fn</span> |
| |
| <div class="viewcode-block" id="BatchElements.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.BatchElements.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="k">if</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="o">.</span><span class="n">runner</span><span class="p">,</span> <span class="s1">'is_streaming'</span><span class="p">,</span> <span class="kc">False</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">"Requires stateful processing (BEAM-2687)"</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">windowing</span><span class="o">.</span><span class="n">is_default</span><span class="p">():</span> |
| <span class="c1"># This is the same logic as _GlobalWindowsBatchingDoFn, but optimized</span> |
| <span class="c1"># for that simpler case.</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span> |
| <span class="n">_GlobalWindowsBatchingDoFn</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_element_size_fn</span><span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span> |
| <span class="n">_WindowAwareBatchingDoFn</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_batch_size_estimator</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_element_size_fn</span><span class="p">))</span></div></div> |
| |
| |
| <span class="k">class</span> <span class="nc">_IdentityWindowFn</span><span class="p">(</span><span class="n">NonMergingWindowFn</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Windowing function that preserves existing windows.</span> |
| |
| <span class="sd"> To be used internally with the Reshuffle transform.</span> |
| <span class="sd"> Will raise an exception when used after DoFns that return TimestampedValue</span> |
| <span class="sd"> elements.</span> |
| <span class="sd"> """</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">window_coder</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Create a new WindowFn with compatible coder.</span> |
| <span class="sd"> To be applied to PCollections with windows that are compatible with the</span> |
| <span class="sd"> given coder.</span> |
| |
| <span class="sd"> Arguments:</span> |
| <span class="sd"> window_coder: coders.Coder object to be used on windows.</span> |
| <span class="sd"> """</span> |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span> |
| <span class="k">if</span> <span class="n">window_coder</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">'window_coder should not be None'</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_window_coder</span> <span class="o">=</span> <span class="n">window_coder</span> |
| |
| <span class="k">def</span> <span class="nf">assign</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">assign_context</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">assign_context</span><span class="o">.</span><span class="n">window</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'assign_context.window should not be None. '</span> |
| <span class="s1">'This might be due to a DoFn returning a TimestampedValue.'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="p">[</span><span class="n">assign_context</span><span class="o">.</span><span class="n">window</span><span class="p">]</span> |
| |
| <span class="k">def</span> <span class="nf">get_window_coder</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">_window_coder</span> |
| |
| |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="k">class</span> <span class="nc">ReshufflePerKey</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransform that returns a PCollection equivalent to its input,</span> |
| <span class="sd"> but operationally provides some of the side effects of a GroupByKey,</span> |
| <span class="sd"> in particular checkpointing, and preventing fusion of the surrounding</span> |
| <span class="sd"> transforms.</span> |
| |
| <span class="sd"> ReshufflePerKey is experimental. No backwards compatibility guarantees.</span> |
| <span class="sd"> """</span> |
| <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="n">windowing_saved</span> <span class="o">=</span> <span class="n">pcoll</span><span class="o">.</span><span class="n">windowing</span> |
| <span class="k">if</span> <span class="n">windowing_saved</span><span class="o">.</span><span class="n">is_default</span><span class="p">():</span> |
| <span class="c1"># In this (common) case we can use a trivial trigger driver</span> |
| <span class="c1"># and avoid the (expensive) window param.</span> |
| <span class="n">globally_windowed</span> <span class="o">=</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="kc">None</span><span class="p">)</span> |
| <span class="n">MIN_TIMESTAMP</span> <span class="o">=</span> <span class="n">window</span><span class="o">.</span><span class="n">MIN_TIMESTAMP</span> |
| |
| <span class="k">def</span> <span class="nf">reify_timestamps</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">):</span> |
| <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">element</span> |
| <span class="k">if</span> <span class="n">timestamp</span> <span class="o">==</span> <span class="n">MIN_TIMESTAMP</span><span class="p">:</span> |
| <span class="n">timestamp</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">return</span> <span class="n">key</span><span class="p">,</span> <span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">restore_timestamps</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">key</span><span class="p">,</span> <span class="n">values</span> <span class="o">=</span> <span class="n">element</span> |
| <span class="k">return</span> <span class="p">[</span> |
| <span class="n">globally_windowed</span><span class="o">.</span><span class="n">with_value</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">))</span> <span class="k">if</span> <span class="n">timestamp</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</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="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">),</span> <span class="n">timestamp</span><span class="p">)</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span> <span class="ow">in</span> <span class="n">values</span> |
| <span class="p">]</span> |
| <span class="k">else</span><span class="p">:</span> |
| |
| <span class="c1"># typing: All conditional function variants must have identical signatures</span> |
| <span class="k">def</span> <span class="nf">reify_timestamps</span><span class="p">(</span> <span class="c1"># type: ignore[misc]</span> |
| <span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">):</span> |
| <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">element</span> |
| <span class="c1"># Transport the window as part of the value and restore it later.</span> |
| <span class="k">return</span> <span class="n">key</span><span class="p">,</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="p">[</span><span class="n">window</span><span class="p">])</span> |
| |
| <span class="k">def</span> <span class="nf">restore_timestamps</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">key</span><span class="p">,</span> <span class="n">windowed_values</span> <span class="o">=</span> <span class="n">element</span> |
| <span class="k">return</span> <span class="p">[</span><span class="n">wv</span><span class="o">.</span><span class="n">with_value</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="n">wv</span><span class="o">.</span><span class="n">value</span><span class="p">))</span> <span class="k">for</span> <span class="n">wv</span> <span class="ow">in</span> <span class="n">windowed_values</span><span class="p">]</span> |
| |
| <span class="n">ungrouped</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="n">reify_timestamps</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Any</span><span class="p">)</span> |
| |
| <span class="c1"># TODO(https://github.com/apache/beam/issues/19785) Using global window as</span> |
| <span class="c1"># one of the standard window. This is to mitigate the Dataflow Java Runner</span> |
| <span class="c1"># Harness limitation to accept only standard coders.</span> |
| <span class="n">ungrouped</span><span class="o">.</span><span class="n">_windowing</span> <span class="o">=</span> <span class="n">Windowing</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">triggerfn</span><span class="o">=</span><span class="n">Always</span><span class="p">(),</span> |
| <span class="n">accumulation_mode</span><span class="o">=</span><span class="n">AccumulationMode</span><span class="o">.</span><span class="n">DISCARDING</span><span class="p">,</span> |
| <span class="n">timestamp_combiner</span><span class="o">=</span><span class="n">TimestampCombiner</span><span class="o">.</span><span class="n">OUTPUT_AT_EARLIEST</span><span class="p">)</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">ungrouped</span> |
| <span class="o">|</span> <span class="n">GroupByKey</span><span class="p">()</span> |
| <span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">restore_timestamps</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Any</span><span class="p">))</span> |
| <span class="n">result</span><span class="o">.</span><span class="n">_windowing</span> <span class="o">=</span> <span class="n">windowing_saved</span> |
| <span class="k">return</span> <span class="n">result</span> |
| |
| |
| <div class="viewcode-block" id="Reshuffle"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reshuffle">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">Reshuffle</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransform that returns a PCollection equivalent to its input,</span> |
| <span class="sd"> but operationally provides some of the side effects of a GroupByKey,</span> |
| <span class="sd"> in particular checkpointing, and preventing fusion of the surrounding</span> |
| <span class="sd"> transforms.</span> |
| |
| <span class="sd"> Reshuffle adds a temporary random key to each element, performs a</span> |
| <span class="sd"> ReshufflePerKey, and finally removes the temporary key.</span> |
| |
| <span class="sd"> Reshuffle is experimental. No backwards compatibility guarantees.</span> |
| <span class="sd"> """</span> |
| |
| <span class="c1"># We use 32-bit integer as the default number of buckets.</span> |
| <span class="n">_DEFAULT_NUM_BUCKETS</span> <span class="o">=</span> <span class="mi">1</span> <span class="o"><<</span> <span class="mi">32</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">num_buckets</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> :param num_buckets: If set, specifies the maximum random keys that would be</span> |
| <span class="sd"> generated.</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">num_buckets</span> <span class="o">=</span> <span class="n">num_buckets</span> <span class="k">if</span> <span class="n">num_buckets</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">_DEFAULT_NUM_BUCKETS</span> |
| |
| <span class="n">valid_buckets</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">num_buckets</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="ow">and</span> <span class="n">num_buckets</span> <span class="o">></span> <span class="mi">0</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="p">(</span><span class="n">num_buckets</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">valid_buckets</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'If `num_buckets` is set, it has to be an '</span> |
| <span class="s1">'integer greater than 0, got </span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="n">num_buckets</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="Reshuffle.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reshuffle.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="c1"># type: (pvalue.PValue) -> pvalue.PCollection</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">pcoll</span> <span class="o">|</span> <span class="s1">'AddRandomKeys'</span> <span class="o">>></span> |
| <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">t</span><span class="p">:</span> <span class="p">(</span><span class="n">random</span><span class="o">.</span><span class="n">randrange</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">num_buckets</span><span class="p">),</span> <span class="n">t</span><span class="p">)</span> |
| <span class="p">)</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">T</span><span class="p">])</span> |
| <span class="o">|</span> <span class="n">ReshufflePerKey</span><span class="p">()</span> |
| <span class="o">|</span> <span class="s1">'RemoveRandomKeys'</span> <span class="o">>></span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">t</span><span class="p">:</span> <span class="n">t</span><span class="p">[</span><span class="mi">1</span><span class="p">])</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span> |
| <span class="n">Tuple</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">T</span><span class="p">])</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="Reshuffle.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reshuffle.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span> |
| <span class="c1"># type: (PipelineContext) -> Tuple[str, None]</span> |
| <span class="k">return</span> <span class="n">common_urns</span><span class="o">.</span><span class="n">composites</span><span class="o">.</span><span class="n">RESHUFFLE</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span></div> |
| |
| <div class="viewcode-block" id="Reshuffle.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reshuffle.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span><span class="n">common_urns</span><span class="o">.</span><span class="n">composites</span><span class="o">.</span><span class="n">RESHUFFLE</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span> |
| <span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">unused_parameter</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">Reshuffle</span><span class="p">()</span></div></div> |
| |
| |
| <span class="k">def</span> <span class="nf">fn_takes_side_inputs</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span> |
| <span class="n">fn</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="s1">'_argspec_fn'</span><span class="p">,</span> <span class="n">fn</span><span class="p">)</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">signature</span> <span class="o">=</span> <span class="n">get_signature</span><span class="p">(</span><span class="n">fn</span><span class="p">)</span> |
| <span class="k">except</span> <span class="ne">TypeError</span><span class="p">:</span> |
| <span class="c1"># We can't tell; maybe it does.</span> |
| <span class="k">return</span> <span class="kc">True</span> |
| |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="nb">len</span><span class="p">(</span><span class="n">signature</span><span class="o">.</span><span class="n">parameters</span><span class="p">)</span> <span class="o">></span> <span class="mi">1</span> <span class="ow">or</span> <span class="nb">any</span><span class="p">(</span> |
| <span class="n">p</span><span class="o">.</span><span class="n">kind</span> <span class="o">==</span> <span class="n">p</span><span class="o">.</span><span class="n">VAR_POSITIONAL</span> <span class="ow">or</span> <span class="n">p</span><span class="o">.</span><span class="n">kind</span> <span class="o">==</span> <span class="n">p</span><span class="o">.</span><span class="n">VAR_KEYWORD</span> |
| <span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="n">signature</span><span class="o">.</span><span class="n">parameters</span><span class="o">.</span><span class="n">values</span><span class="p">()))</span> |
| |
| |
| <div class="viewcode-block" id="WithKeys"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.WithKeys">[docs]</a><span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">WithKeys</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">k</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransform that takes a PCollection, and either a constant key or a</span> |
| <span class="sd"> callable, and returns a PCollection of (K, V), where each of the values in</span> |
| <span class="sd"> the input PCollection has been paired with either the constant key or a key</span> |
| <span class="sd"> computed from the value. The callable may optionally accept positional or</span> |
| <span class="sd"> keyword arguments, which should be passed to WithKeys directly. These may</span> |
| <span class="sd"> be either SideInputs or static (non-PCollection) values, such as ints.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="nb">callable</span><span class="p">(</span><span class="n">k</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">fn_takes_side_inputs</span><span class="p">(</span><span class="n">k</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">arg</span><span class="p">,</span> <span class="n">AsSideInput</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">args</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">kwarg</span><span class="p">,</span> <span class="n">AsSideInput</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">kwarg</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">values</span><span class="p">()):</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">v</span><span class="p">,</span> |
| <span class="o">*</span><span class="n">args</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="p">(</span><span class="n">k</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="n">v</span><span class="p">),</span> |
| <span class="o">*</span><span class="n">args</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">k</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="n">v</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">k</span><span class="p">(</span><span class="n">v</span><span class="p">),</span> <span class="n">v</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">))</span></div> |
| |
| |
| <div class="viewcode-block" id="GroupIntoBatches"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.GroupIntoBatches">[docs]</a><span class="nd">@experimental</span><span class="p">()</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">Iterable</span><span class="p">[</span><span class="n">V</span><span class="p">]])</span> |
| <span class="k">class</span> <span class="nc">GroupIntoBatches</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransform that batches the input into desired batch size. Elements are</span> |
| <span class="sd"> buffered until they are equal to batch size provided in the argument at which</span> |
| <span class="sd"> point they are output to the output Pcollection.</span> |
| |
| <span class="sd"> Windows are preserved (batches will contain elements from the same window)</span> |
| |
| <span class="sd"> GroupIntoBatches is experimental. Its use case will depend on the runner if</span> |
| <span class="sd"> it has support of States and Timers.</span> |
| <span class="sd"> """</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">batch_size</span><span class="p">,</span> <span class="n">max_buffering_duration_secs</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">clock</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="w"> </span><span class="sd">"""Create a new GroupIntoBatches.</span> |
| |
| <span class="sd"> Arguments:</span> |
| <span class="sd"> batch_size: (required) How many elements should be in a batch</span> |
| <span class="sd"> max_buffering_duration_secs: (optional) How long in seconds at most an</span> |
| <span class="sd"> incomplete batch of elements is allowed to be buffered in the states.</span> |
| <span class="sd"> The duration must be a positive second duration and should be given as</span> |
| <span class="sd"> an int or float. Setting this parameter to zero effectively means no</span> |
| <span class="sd"> buffering limit.</span> |
| <span class="sd"> clock: (optional) an alternative to time.time (mostly for testing)</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">params</span> <span class="o">=</span> <span class="n">_GroupIntoBatchesParams</span><span class="p">(</span> |
| <span class="n">batch_size</span><span class="p">,</span> <span class="n">max_buffering_duration_secs</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">clock</span> <span class="o">=</span> <span class="n">clock</span> |
| |
| <div class="viewcode-block" id="GroupIntoBatches.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.GroupIntoBatches.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="n">input_coder</span> <span class="o">=</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">pcoll</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span> |
| <span class="n">_pardo_group_into_batches</span><span class="p">(</span> |
| <span class="n">input_coder</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">params</span><span class="o">.</span><span class="n">batch_size</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">params</span><span class="o">.</span><span class="n">max_buffering_duration_secs</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">clock</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="GroupIntoBatches.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.GroupIntoBatches.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">unused_context</span> <span class="c1"># type: PipelineContext</span> |
| <span class="p">):</span> <span class="c1"># type: (...) -> Tuple[str, beam_runner_api_pb2.GroupIntoBatchesPayload]</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">common_urns</span><span class="o">.</span><span class="n">group_into_batches_components</span><span class="o">.</span><span class="n">GROUP_INTO_BATCHES</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">params</span><span class="o">.</span><span class="n">get_payload</span><span class="p">())</span></div> |
| |
| <div class="viewcode-block" id="GroupIntoBatches.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.GroupIntoBatches.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span> |
| <span class="n">common_urns</span><span class="o">.</span><span class="n">group_into_batches_components</span><span class="o">.</span><span class="n">GROUP_INTO_BATCHES</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> |
| <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">GroupIntoBatchesPayload</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">proto</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">GroupIntoBatches</span><span class="p">(</span><span class="o">*</span><span class="n">_GroupIntoBatchesParams</span><span class="o">.</span><span class="n">parse_payload</span><span class="p">(</span><span class="n">proto</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="GroupIntoBatches.WithShardedKey"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.GroupIntoBatches.WithShardedKey">[docs]</a> <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span> |
| <span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span> |
| <span class="n">ShardedKeyType</span><span class="p">[</span><span class="n">typehints</span><span class="o">.</span><span class="n">TypeVariable</span><span class="p">(</span><span class="n">K</span><span class="p">)],</span> <span class="c1"># type: ignore[misc]</span> |
| <span class="n">typehints</span><span class="o">.</span><span class="n">Iterable</span><span class="p">[</span><span class="n">typehints</span><span class="o">.</span><span class="n">TypeVariable</span><span class="p">(</span><span class="n">V</span><span class="p">)]])</span> |
| <span class="k">class</span> <span class="nc">WithShardedKey</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A GroupIntoBatches transform that outputs batched elements associated</span> |
| <span class="sd"> with sharded input keys.</span> |
| |
| <span class="sd"> By default, keys are sharded to such that the input elements with the same</span> |
| <span class="sd"> key are spread to all available threads executing the transform. Runners may</span> |
| <span class="sd"> override the default sharding to do a better load balancing during the</span> |
| <span class="sd"> execution time.</span> |
| <span class="sd"> """</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">batch_size</span><span class="p">,</span> <span class="n">max_buffering_duration_secs</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">clock</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="w"> </span><span class="sd">"""Create a new GroupIntoBatches with sharded output.</span> |
| <span class="sd"> See ``GroupIntoBatches`` transform for a description of input parameters.</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">params</span> <span class="o">=</span> <span class="n">_GroupIntoBatchesParams</span><span class="p">(</span> |
| <span class="n">batch_size</span><span class="p">,</span> <span class="n">max_buffering_duration_secs</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">clock</span> <span class="o">=</span> <span class="n">clock</span> |
| |
| <span class="n">_shard_id_prefix</span> <span class="o">=</span> <span class="n">uuid</span><span class="o">.</span><span class="n">uuid4</span><span class="p">()</span><span class="o">.</span><span class="n">bytes</span> |
| |
| <div class="viewcode-block" id="GroupIntoBatches.WithShardedKey.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.GroupIntoBatches.WithShardedKey.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="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="n">sharded_pcoll</span> <span class="o">=</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">key_value</span><span class="p">:</span> <span class="p">(</span> |
| <span class="n">ShardedKey</span><span class="p">(</span> |
| <span class="n">key_value</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> |
| <span class="c1"># Use [uuid, thread id] as the shard id.</span> |
| <span class="n">GroupIntoBatches</span><span class="o">.</span><span class="n">WithShardedKey</span><span class="o">.</span><span class="n">_shard_id_prefix</span> <span class="o">+</span> <span class="nb">bytes</span><span class="p">(</span> |
| <span class="n">threading</span><span class="o">.</span><span class="n">get_ident</span><span class="p">()</span><span class="o">.</span><span class="n">to_bytes</span><span class="p">(</span><span class="mi">8</span><span class="p">,</span> <span class="s1">'big'</span><span class="p">))),</span> |
| <span class="n">key_value</span><span class="p">[</span><span class="mi">1</span><span class="p">]))</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span> |
| <span class="n">typehints</span><span class="o">.</span><span class="n">Tuple</span><span class="p">[</span> |
| <span class="n">ShardedKeyType</span><span class="p">[</span><span class="n">key_type</span><span class="p">],</span> <span class="c1"># type: ignore[misc]</span> |
| <span class="n">value_type</span><span class="p">])</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">sharded_pcoll</span> |
| <span class="o">|</span> <span class="n">GroupIntoBatches</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">params</span><span class="o">.</span><span class="n">batch_size</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">params</span><span class="o">.</span><span class="n">max_buffering_duration_secs</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">clock</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="GroupIntoBatches.WithShardedKey.to_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.GroupIntoBatches.WithShardedKey.to_runner_api_parameter">[docs]</a> <span class="k">def</span> <span class="nf">to_runner_api_parameter</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">unused_context</span> <span class="c1"># type: PipelineContext</span> |
| <span class="p">):</span> <span class="c1"># type: (...) -> Tuple[str, beam_runner_api_pb2.GroupIntoBatchesPayload]</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">common_urns</span><span class="o">.</span><span class="n">composites</span><span class="o">.</span><span class="n">GROUP_INTO_BATCHES_WITH_SHARDED_KEY</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">params</span><span class="o">.</span><span class="n">get_payload</span><span class="p">())</span></div> |
| |
| <div class="viewcode-block" id="GroupIntoBatches.WithShardedKey.from_runner_api_parameter"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.GroupIntoBatches.WithShardedKey.from_runner_api_parameter">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@PTransform</span><span class="o">.</span><span class="n">register_urn</span><span class="p">(</span> |
| <span class="n">common_urns</span><span class="o">.</span><span class="n">composites</span><span class="o">.</span><span class="n">GROUP_INTO_BATCHES_WITH_SHARDED_KEY</span><span class="o">.</span><span class="n">urn</span><span class="p">,</span> |
| <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">GroupIntoBatchesPayload</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">from_runner_api_parameter</span><span class="p">(</span><span class="n">unused_ptransform</span><span class="p">,</span> <span class="n">proto</span><span class="p">,</span> <span class="n">unused_context</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">GroupIntoBatches</span><span class="o">.</span><span class="n">WithShardedKey</span><span class="p">(</span> |
| <span class="o">*</span><span class="n">_GroupIntoBatchesParams</span><span class="o">.</span><span class="n">parse_payload</span><span class="p">(</span><span class="n">proto</span><span class="p">))</span></div></div></div> |
| |
| |
| <span class="k">class</span> <span class="nc">_GroupIntoBatchesParams</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""This class represents the parameters for</span> |
| <span class="sd"> :class:`apache_beam.utils.GroupIntoBatches` transform, used to define how</span> |
| <span class="sd"> elements should be batched.</span> |
| <span class="sd"> """</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">batch_size</span><span class="p">,</span> <span class="n">max_buffering_duration_secs</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">batch_size</span> <span class="o">=</span> <span class="n">batch_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">max_buffering_duration_secs</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="mi">0</span> |
| <span class="k">if</span> <span class="n">max_buffering_duration_secs</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="n">max_buffering_duration_secs</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_validate</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">other</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">_GroupIntoBatchesParams</span><span class="p">):</span> |
| <span class="k">return</span> <span class="kc">False</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">batch_size</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">batch_size</span> <span class="ow">and</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">max_buffering_duration_secs</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">max_buffering_duration_secs</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_validate</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">assert</span> <span class="bp">self</span><span class="o">.</span><span class="n">batch_size</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">batch_size</span> <span class="o">></span> <span class="mi">0</span><span class="p">,</span> <span class="p">(</span> |
| <span class="s1">'batch_size must be a positive value'</span><span class="p">)</span> |
| <span class="k">assert</span> <span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">max_buffering_duration_secs</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">max_buffering_duration_secs</span> <span class="o">>=</span> <span class="mi">0</span><span class="p">),</span> <span class="p">(</span> |
| <span class="s1">'max_buffering_duration must be a non-negative value'</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">get_payload</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">GroupIntoBatchesPayload</span><span class="p">(</span> |
| <span class="n">batch_size</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">batch_size</span><span class="p">,</span> |
| <span class="n">max_buffering_duration_millis</span><span class="o">=</span><span class="nb">int</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">max_buffering_duration_secs</span> <span class="o">*</span> <span class="mi">1000</span><span class="p">))</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">parse_payload</span><span class="p">(</span> |
| <span class="n">proto</span> <span class="c1"># type: beam_runner_api_pb2.GroupIntoBatchesPayload</span> |
| <span class="p">):</span> |
| <span class="k">return</span> <span class="n">proto</span><span class="o">.</span><span class="n">batch_size</span><span class="p">,</span> <span class="n">proto</span><span class="o">.</span><span class="n">max_buffering_duration_millis</span> <span class="o">/</span> <span class="mi">1000</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_pardo_group_into_batches</span><span class="p">(</span> |
| <span class="n">input_coder</span><span class="p">,</span> <span class="n">batch_size</span><span class="p">,</span> <span class="n">max_buffering_duration_secs</span><span class="p">,</span> <span class="n">clock</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">ELEMENT_STATE</span> <span class="o">=</span> <span class="n">BagStateSpec</span><span class="p">(</span><span class="s1">'values'</span><span class="p">,</span> <span class="n">input_coder</span><span class="p">)</span> |
| <span class="n">COUNT_STATE</span> <span class="o">=</span> <span class="n">CombiningValueStateSpec</span><span class="p">(</span><span class="s1">'count'</span><span class="p">,</span> <span class="n">input_coder</span><span class="p">,</span> <span class="n">CountCombineFn</span><span class="p">())</span> |
| <span class="n">WINDOW_TIMER</span> <span class="o">=</span> <span class="n">TimerSpec</span><span class="p">(</span><span class="s1">'window_end'</span><span class="p">,</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">WATERMARK</span><span class="p">)</span> |
| <span class="n">BUFFERING_TIMER</span> <span class="o">=</span> <span class="n">TimerSpec</span><span class="p">(</span><span class="s1">'buffering_end'</span><span class="p">,</span> <span class="n">TimeDomain</span><span class="o">.</span><span class="n">REAL_TIME</span><span class="p">)</span> |
| |
| <span class="k">class</span> <span class="nc">_GroupIntoBatchesDoFn</span><span class="p">(</span><span class="n">DoFn</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">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">,</span> |
| <span class="n">element_state</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">ELEMENT_STATE</span><span class="p">),</span> |
| <span class="n">count_state</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">window_timer</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimerParam</span><span class="p">(</span><span class="n">WINDOW_TIMER</span><span class="p">),</span> |
| <span class="n">buffering_timer</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimerParam</span><span class="p">(</span><span class="n">BUFFERING_TIMER</span><span class="p">)):</span> |
| <span class="c1"># Allowed lateness not supported in Python SDK</span> |
| <span class="c1"># https://beam.apache.org/documentation/programming-guide/#watermarks-and-late-data</span> |
| <span class="n">window_timer</span><span class="o">.</span><span class="n">set</span><span class="p">(</span><span class="n">window</span><span class="o">.</span><span class="n">end</span><span class="p">)</span> |
| <span class="n">element_state</span><span class="o">.</span><span class="n">add</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">add</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span> |
| <span class="n">count</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">count</span> <span class="o">==</span> <span class="mi">1</span> <span class="ow">and</span> <span class="n">max_buffering_duration_secs</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="c1"># This is the first element in batch. Start counting buffering time if a</span> |
| <span class="c1"># limit was set.</span> |
| <span class="c1"># pylint: disable=deprecated-method</span> |
| <span class="n">buffering_timer</span><span class="o">.</span><span class="n">set</span><span class="p">(</span><span class="n">clock</span><span class="p">()</span> <span class="o">+</span> <span class="n">max_buffering_duration_secs</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">count</span> <span class="o">>=</span> <span class="n">batch_size</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">flush_batch</span><span class="p">(</span><span class="n">element_state</span><span class="p">,</span> <span class="n">count_state</span><span class="p">,</span> <span class="n">buffering_timer</span><span class="p">)</span> |
| |
| <span class="nd">@on_timer</span><span class="p">(</span><span class="n">WINDOW_TIMER</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">on_window_timer</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">element_state</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">ELEMENT_STATE</span><span class="p">),</span> |
| <span class="n">count_state</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">buffering_timer</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimerParam</span><span class="p">(</span><span class="n">BUFFERING_TIMER</span><span class="p">)):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">flush_batch</span><span class="p">(</span><span class="n">element_state</span><span class="p">,</span> <span class="n">count_state</span><span class="p">,</span> <span class="n">buffering_timer</span><span class="p">)</span> |
| |
| <span class="nd">@on_timer</span><span class="p">(</span><span class="n">BUFFERING_TIMER</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">on_buffering_timer</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">element_state</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">ELEMENT_STATE</span><span class="p">),</span> |
| <span class="n">count_state</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">buffering_timer</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimerParam</span><span class="p">(</span><span class="n">BUFFERING_TIMER</span><span class="p">)):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">flush_batch</span><span class="p">(</span><span class="n">element_state</span><span class="p">,</span> <span class="n">count_state</span><span class="p">,</span> <span class="n">buffering_timer</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">flush_batch</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">element_state</span><span class="p">,</span> <span class="n">count_state</span><span class="p">,</span> <span class="n">buffering_timer</span><span class="p">):</span> |
| <span class="n">batch</span> <span class="o">=</span> <span class="p">[</span><span class="n">element</span> <span class="k">for</span> <span class="n">element</span> <span class="ow">in</span> <span class="n">element_state</span><span class="o">.</span><span class="n">read</span><span class="p">()]</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">batch</span><span class="p">:</span> |
| <span class="k">return</span> |
| <span class="n">key</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">batch</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> |
| <span class="n">batch_values</span> <span class="o">=</span> <span class="p">[</span><span class="n">v</span> <span class="k">for</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">)</span> <span class="ow">in</span> <span class="n">batch</span><span class="p">]</span> |
| <span class="n">element_state</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span> |
| <span class="n">count_state</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span> |
| <span class="n">buffering_timer</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span> |
| <span class="k">yield</span> <span class="n">key</span><span class="p">,</span> <span class="n">batch_values</span> |
| |
| <span class="k">return</span> <span class="n">_GroupIntoBatchesDoFn</span><span class="p">()</span> |
| |
| |
| <div class="viewcode-block" id="ToString"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.ToString">[docs]</a><span class="k">class</span> <span class="nc">ToString</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> PTransform for converting a PCollection element, KV or PCollection Iterable</span> |
| <span class="sd"> to string.</span> |
| <span class="sd"> """</span> |
| |
| <span class="c1"># pylint: disable=invalid-name</span> |
| <div class="viewcode-block" id="ToString.Element"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.ToString.Element">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">Element</span><span class="p">():</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Transforms each element of the PCollection to a string.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="s1">'ElementToString'</span> <span class="o">>></span> <span class="n">Map</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="ToString.Iterables"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.ToString.Iterables">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">Iterables</span><span class="p">(</span><span class="n">delimiter</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Transforms each item in the iterable of the input of PCollection to a</span> |
| <span class="sd"> string. There is no trailing delimiter.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="n">delimiter</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">delimiter</span> <span class="o">=</span> <span class="s1">','</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="s1">'IterablesToString'</span> <span class="o">>></span> |
| <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">xs</span><span class="p">:</span> <span class="n">delimiter</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">xs</span><span class="p">))</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span> |
| <span class="n">Iterable</span><span class="p">[</span><span class="n">Any</span><span class="p">])</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="nb">str</span><span class="p">))</span></div> |
| |
| <span class="c1"># An alias for Iterables.</span> |
| <span class="n">Kvs</span> <span class="o">=</span> <span class="n">Iterables</span></div> |
| |
| |
| <div class="viewcode-block" id="LogElements"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.LogElements">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">LogElements</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> PTransform for printing the elements of a PCollection.</span> |
| <span class="sd"> """</span> |
| <span class="k">class</span> <span class="nc">_LoggingFn</span><span class="p">(</span><span class="n">DoFn</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">prefix</span><span class="o">=</span><span class="s1">''</span><span class="p">,</span> <span class="n">with_timestamp</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">with_window</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">prefix</span> <span class="o">=</span> <span class="n">prefix</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">with_timestamp</span> <span class="o">=</span> <span class="n">with_timestamp</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">with_window</span> <span class="o">=</span> <span class="n">with_window</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">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">,</span> |
| <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="n">log_line</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">prefix</span> <span class="o">+</span> <span class="nb">str</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">with_timestamp</span><span class="p">:</span> |
| <span class="n">log_line</span> <span class="o">+=</span> <span class="s1">', timestamp='</span> <span class="o">+</span> <span class="nb">repr</span><span class="p">(</span><span class="n">timestamp</span><span class="o">.</span><span class="n">to_rfc3339</span><span class="p">())</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">with_window</span><span class="p">:</span> |
| <span class="n">log_line</span> <span class="o">+=</span> <span class="s1">', window(start='</span> <span class="o">+</span> <span class="n">window</span><span class="o">.</span><span class="n">start</span><span class="o">.</span><span class="n">to_rfc3339</span><span class="p">()</span> |
| <span class="n">log_line</span> <span class="o">+=</span> <span class="s1">', end='</span> <span class="o">+</span> <span class="n">window</span><span class="o">.</span><span class="n">end</span><span class="o">.</span><span class="n">to_rfc3339</span><span class="p">()</span> <span class="o">+</span> <span class="s1">')'</span> |
| |
| <span class="nb">print</span><span class="p">(</span><span class="n">log_line</span><span class="p">)</span> |
| <span class="k">yield</span> <span class="n">element</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">label</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">prefix</span><span class="o">=</span><span class="s1">''</span><span class="p">,</span> <span class="n">with_timestamp</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">with_window</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">prefix</span> <span class="o">=</span> <span class="n">prefix</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">with_timestamp</span> <span class="o">=</span> <span class="n">with_timestamp</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">with_window</span> <span class="o">=</span> <span class="n">with_window</span> |
| |
| <div class="viewcode-block" id="LogElements.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.LogElements.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="nb">input</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">input</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_LoggingFn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">prefix</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">with_timestamp</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">with_window</span><span class="p">))</span></div></div> |
| |
| |
| <div class="viewcode-block" id="Reify"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify">[docs]</a><span class="k">class</span> <span class="nc">Reify</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransforms for converting between explicit and implicit form of various</span> |
| <span class="sd"> Beam values."""</span> |
| <div class="viewcode-block" id="Reify.Timestamp"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.Timestamp">[docs]</a> <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">Timestamp</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransform to wrap a value in a TimestampedValue with it's</span> |
| <span class="sd"> associated timestamp."""</span> |
| <div class="viewcode-block" id="Reify.Timestamp.add_timestamp_info"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.Timestamp.add_timestamp_info">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">add_timestamp_info</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">TimestampedValue</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Reify.Timestamp.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.Timestamp.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="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">add_timestamp_info</span><span class="p">)</span></div></div> |
| |
| <div class="viewcode-block" id="Reify.Window"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.Window">[docs]</a> <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">Window</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransform to convert an element in a PCollection into a tuple of</span> |
| <span class="sd"> (element, timestamp, window), wrapped in a TimestampedValue with it's</span> |
| <span class="sd"> associated timestamp."""</span> |
| <div class="viewcode-block" id="Reify.Window.add_window_info"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.Window.add_window_info">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">add_window_info</span><span class="p">(</span> |
| <span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">TimestampedValue</span><span class="p">((</span><span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">window</span><span class="p">),</span> <span class="n">timestamp</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Reify.Window.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.Window.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="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">add_window_info</span><span class="p">)</span></div></div> |
| |
| <div class="viewcode-block" id="Reify.TimestampInValue"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.TimestampInValue">[docs]</a> <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="k">class</span> <span class="nc">TimestampInValue</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransform to wrap the Value in a KV pair in a TimestampedValue with</span> |
| <span class="sd"> the element's associated timestamp."""</span> |
| <div class="viewcode-block" id="Reify.TimestampInValue.add_timestamp_info"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.TimestampInValue.add_timestamp_info">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">add_timestamp_info</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">):</span> |
| <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">element</span> |
| <span class="k">yield</span> <span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">TimestampedValue</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="Reify.TimestampInValue.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.TimestampInValue.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="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">add_timestamp_info</span><span class="p">)</span></div></div> |
| |
| <div class="viewcode-block" id="Reify.WindowInValue"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.WindowInValue">[docs]</a> <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">K</span><span class="p">,</span> <span class="n">V</span><span class="p">])</span> |
| <span class="k">class</span> <span class="nc">WindowInValue</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""PTransform to convert the Value in a KV pair into a tuple of</span> |
| <span class="sd"> (value, timestamp, window), with the whole element being wrapped inside a</span> |
| <span class="sd"> TimestampedValue."""</span> |
| <div class="viewcode-block" id="Reify.WindowInValue.add_window_info"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.WindowInValue.add_window_info">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">add_window_info</span><span class="p">(</span> |
| <span class="n">element</span><span class="p">,</span> <span class="n">timestamp</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">,</span> <span class="n">window</span><span class="o">=</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">):</span> |
| <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="o">=</span> <span class="n">element</span> |
| <span class="k">yield</span> <span class="n">TimestampedValue</span><span class="p">((</span><span class="n">key</span><span class="p">,</span> <span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">,</span> <span class="n">window</span><span class="p">)),</span> <span class="n">timestamp</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Reify.WindowInValue.expand"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Reify.WindowInValue.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="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">add_window_info</span><span class="p">)</span></div></div></div> |
| |
| |
| <div class="viewcode-block" id="Regex"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex">[docs]</a><span class="k">class</span> <span class="nc">Regex</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> PTransform to use Regular Expression to process the elements in a</span> |
| <span class="sd"> PCollection.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">ALL</span> <span class="o">=</span> <span class="s2">"__regex_all_groups"</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Return re.compile if the regex has a string value"""</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">regex</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">regex</span> |
| |
| <div class="viewcode-block" id="Regex.matches"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.matches">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">matches</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">,</span> <span class="n">group</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns the matches (group 0 by default) if zero or more characters at the</span> |
| <span class="sd"> beginning of string match the regular expression. To match the entire</span> |
| <span class="sd"> string, add "$" sign at the end of regex expression.</span> |
| |
| <span class="sd"> Group can be integer value or a string value.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> group: (optional) name/number of the group, it can be integer or a string</span> |
| <span class="sd"> value. Defaults to 0, meaning the entire matched string will be</span> |
| <span class="sd"> returned.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_process</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">m</span> <span class="o">=</span> <span class="n">regex</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">m</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">group</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">_process</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Regex.all_matches"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.all_matches">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">])</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">all_matches</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns all matches (groups) if zero or more characters at the beginning</span> |
| <span class="sd"> of string match the regular expression.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_process</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">m</span> <span class="o">=</span> <span class="n">regex</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">m</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="p">[</span><span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">ix</span><span class="p">)</span> <span class="k">for</span> <span class="n">ix</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">m</span><span class="o">.</span><span class="n">lastindex</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)]</span> |
| |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">_process</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Regex.matches_kv"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.matches_kv">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">str</span><span class="p">])</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">matches_kv</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">,</span> <span class="n">keyGroup</span><span class="p">,</span> <span class="n">valueGroup</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns the KV pairs if the string matches the regular expression, deriving</span> |
| <span class="sd"> the key & value from the specified group of the regular expression.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> keyGroup: The Regex group to use as the key. Can be int or str.</span> |
| <span class="sd"> valueGroup: (optional) Regex group to use the value. Can be int or str.</span> |
| <span class="sd"> The default value "0" returns entire matched string.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_process</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">match</span> <span class="o">=</span> <span class="n">regex</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">match</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="p">(</span><span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">keyGroup</span><span class="p">),</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">valueGroup</span><span class="p">))</span> |
| |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">_process</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Regex.find"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.find">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">find</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">,</span> <span class="n">group</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns the matches if a portion of the line matches the Regex. Returns</span> |
| <span class="sd"> the entire group (group 0 by default). Group can be integer value or a</span> |
| <span class="sd"> string value.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> group: (optional) name of the group, it can be integer or a string value.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_process</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">r</span> <span class="o">=</span> <span class="n">regex</span><span class="o">.</span><span class="n">search</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">r</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">r</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">group</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">_process</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Regex.find_all"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.find_all">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Union</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">],</span> <span class="n">List</span><span class="p">[</span><span class="n">Tuple</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">str</span><span class="p">]]])</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">find_all</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">,</span> <span class="n">group</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">outputEmpty</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns the matches if a portion of the line matches the Regex. By default,</span> |
| <span class="sd"> list of group 0 will return with empty items. To get all groups, pass the</span> |
| <span class="sd"> `Regex.ALL` flag in the `group` parameter which returns all the groups in</span> |
| <span class="sd"> the tuple format.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> group: (optional) name of the group, it can be integer or a string value.</span> |
| <span class="sd"> outputEmpty: (optional) Should empty be output. True to output empties</span> |
| <span class="sd"> and false if not.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_process</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">matches</span> <span class="o">=</span> <span class="n">regex</span><span class="o">.</span><span class="n">finditer</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">group</span> <span class="o">==</span> <span class="n">Regex</span><span class="o">.</span><span class="n">ALL</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="p">[(</span><span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(),</span> <span class="n">m</span><span class="o">.</span><span class="n">groups</span><span class="p">()[</span><span class="mi">0</span><span class="p">])</span> <span class="k">for</span> <span class="n">m</span> <span class="ow">in</span> <span class="n">matches</span> |
| <span class="k">if</span> <span class="n">outputEmpty</span> <span class="ow">or</span> <span class="n">m</span><span class="o">.</span><span class="n">groups</span><span class="p">()[</span><span class="mi">0</span><span class="p">]]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="p">[</span><span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">group</span><span class="p">)</span> <span class="k">for</span> <span class="n">m</span> <span class="ow">in</span> <span class="n">matches</span> <span class="k">if</span> <span class="n">outputEmpty</span> <span class="ow">or</span> <span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">group</span><span class="p">)]</span> |
| |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">_process</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Regex.find_kv"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.find_kv">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">str</span><span class="p">])</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">find_kv</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">,</span> <span class="n">keyGroup</span><span class="p">,</span> <span class="n">valueGroup</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns the matches if a portion of the line matches the Regex. Returns the</span> |
| <span class="sd"> specified groups as the key and value pair.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> keyGroup: The Regex group to use as the key. Can be int or str.</span> |
| <span class="sd"> valueGroup: (optional) Regex group to use the value. Can be int or str.</span> |
| <span class="sd"> The default value "0" returns entire matched string.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_process</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">matches</span> <span class="o">=</span> <span class="n">regex</span><span class="o">.</span><span class="n">finditer</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">matches</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">match</span> <span class="ow">in</span> <span class="n">matches</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="p">(</span><span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">keyGroup</span><span class="p">),</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="n">valueGroup</span><span class="p">))</span> |
| |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">_process</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="Regex.replace_all"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.replace_all">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">replace_all</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">,</span> <span class="n">replacement</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns the matches if a portion of the line matches the regex and</span> |
| <span class="sd"> replaces all matches with the replacement string.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> replacement: the string to be substituted for each match.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">elem</span><span class="p">:</span> <span class="n">regex</span><span class="o">.</span><span class="n">sub</span><span class="p">(</span><span class="n">replacement</span><span class="p">,</span> <span class="n">elem</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="Regex.replace_first"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.replace_first">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">replace_first</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">,</span> <span class="n">replacement</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns the matches if a portion of the line matches the regex and replaces</span> |
| <span class="sd"> the first match with the replacement string.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> replacement: the string to be substituted for each match.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">elem</span><span class="p">:</span> <span class="n">regex</span><span class="o">.</span><span class="n">sub</span><span class="p">(</span><span class="n">replacement</span><span class="p">,</span> <span class="n">elem</span><span class="p">,</span> <span class="mi">1</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="Regex.split"><a class="viewcode-back" href="../../../apache_beam.transforms.util.html#apache_beam.transforms.util.Regex.split">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="nb">str</span><span class="p">)</span> |
| <span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">])</span> |
| <span class="nd">@ptransform_fn</span> |
| <span class="k">def</span> <span class="nf">split</span><span class="p">(</span><span class="n">pcoll</span><span class="p">,</span> <span class="n">regex</span><span class="p">,</span> <span class="n">outputEmpty</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Returns the list string which was splitted on the basis of regular</span> |
| <span class="sd"> expression. It will not output empty items (by defaults).</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> regex: the regular expression string or (re.compile) pattern.</span> |
| <span class="sd"> outputEmpty: (optional) Should empty be output. True to output empties</span> |
| <span class="sd"> and false if not.</span> |
| <span class="sd"> """</span> |
| <span class="n">regex</span> <span class="o">=</span> <span class="n">Regex</span><span class="o">.</span><span class="n">_regex_compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span> |
| <span class="n">outputEmpty</span> <span class="o">=</span> <span class="nb">bool</span><span class="p">(</span><span class="n">outputEmpty</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_process</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="n">r</span> <span class="o">=</span> <span class="n">regex</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">r</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">outputEmpty</span><span class="p">:</span> |
| <span class="n">r</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">filter</span><span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="n">r</span><span class="p">))</span> |
| <span class="k">yield</span> <span class="n">r</span> |
| |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">FlatMap</span><span class="p">(</span><span class="n">_process</span><span class="p">)</span></div></div> |
| </pre></div> |
| |
| </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/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> |