| |
| |
| <!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.testing.synthetic_pipeline — 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.testing.synthetic_pipeline</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.testing.synthetic_pipeline</h1><div class="highlight"><pre> |
| <span></span><span class="c1">#</span> |
| <span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span> |
| <span class="c1"># contributor license agreements. See the NOTICE file distributed with</span> |
| <span class="c1"># this work for additional information regarding copyright ownership.</span> |
| <span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span> |
| <span class="c1"># (the "License"); you may not use this file except in compliance with</span> |
| <span class="c1"># the License. You may obtain a copy of the License at</span> |
| <span class="c1">#</span> |
| <span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span> |
| <span class="c1">#</span> |
| <span class="c1"># Unless required by applicable law or agreed to in writing, software</span> |
| <span class="c1"># distributed under the License is distributed on an "AS IS" BASIS,</span> |
| <span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span> |
| <span class="c1"># See the License for the specific language governing permissions and</span> |
| <span class="c1"># limitations under the License.</span> |
| <span class="c1">#</span> |
| |
| <span class="sd">"""A set of utilities to write pipelines for performance tests.</span> |
| |
| <span class="sd">This module offers a way to create pipelines using synthetic sources and steps.</span> |
| <span class="sd">Exact shape of the pipeline and the behaviour of sources and steps can be</span> |
| <span class="sd">controlled through arguments. Please see function 'parse_args()' for more</span> |
| <span class="sd">details about the arguments.</span> |
| |
| <span class="sd">Shape of the pipeline is primarily controlled through two arguments. Argument</span> |
| <span class="sd">'steps' can be used to define a list of steps as a JSON string. Argument</span> |
| <span class="sd">'barrier' describes how these steps are separated from each other. Argument</span> |
| <span class="sd">'barrier' can be use to build a pipeline as a series of steps or a tree of</span> |
| <span class="sd">steps with a fanin or a fanout of size 2.</span> |
| |
| <span class="sd">Other arguments describe what gets generated by synthetic sources that produce</span> |
| <span class="sd">data for the pipeline.</span> |
| <span class="sd">"""</span> |
| |
| <span class="c1"># pytype: skip-file</span> |
| |
| <span class="kn">import</span> <span class="nn">argparse</span> |
| <span class="kn">import</span> <span class="nn">json</span> |
| <span class="kn">import</span> <span class="nn">logging</span> |
| <span class="kn">import</span> <span class="nn">math</span> |
| <span class="kn">import</span> <span class="nn">os</span> |
| <span class="kn">import</span> <span class="nn">sys</span> |
| <span class="kn">import</span> <span class="nn">time</span> |
| <span class="kn">from</span> <span class="nn">random</span> <span class="kn">import</span> <span class="n">Random</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Tuple</span> |
| |
| <span class="kn">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">pvalue</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.io</span> <span class="kn">import</span> <span class="n">WriteToText</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io</span> <span class="kn">import</span> <span class="n">iobase</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io</span> <span class="kn">import</span> <span class="n">range_trackers</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io</span> <span class="kn">import</span> <span class="n">restriction_trackers</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io.restriction_trackers</span> <span class="kn">import</span> <span class="n">OffsetRange</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io.restriction_trackers</span> <span class="kn">import</span> <span class="n">OffsetRestrictionTracker</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="kn">import</span> <span class="n">PipelineOptions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.options.pipeline_options</span> <span class="kn">import</span> <span class="n">SetupOptions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.testing.test_pipeline</span> <span class="kn">import</span> <span class="n">TestPipeline</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">userstate</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms.core</span> <span class="kn">import</span> <span class="n">RestrictionProvider</span> |
| |
| <span class="k">try</span><span class="p">:</span> |
| <span class="kn">import</span> <span class="nn">numpy</span> <span class="k">as</span> <span class="nn">np</span> |
| <span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span> |
| <span class="n">np</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_Random</span><span class="p">(</span><span class="n">Random</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A subclass of `random.Random` from the Python Standard Library that</span> |
| <span class="sd"> provides a method returning random bytes of arbitrary length.</span> |
| <span class="sd"> """</span> |
| |
| <span class="c1"># `numpy.random.RandomState` does not provide `random()` method, we keep this</span> |
| <span class="c1"># for compatibility reasons.</span> |
| <span class="n">random_sample</span> <span class="o">=</span> <span class="n">Random</span><span class="o">.</span><span class="n">random</span> |
| |
| <span class="k">def</span> <span class="nf">bytes</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">length</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns random bytes.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> length (int): Number of random bytes.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">getrandbits</span><span class="p">(</span><span class="n">length</span> <span class="o">*</span> <span class="mi">8</span><span class="p">)</span><span class="o">.</span><span class="n">to_bytes</span><span class="p">(</span><span class="n">length</span><span class="p">,</span> <span class="n">sys</span><span class="o">.</span><span class="n">byteorder</span><span class="p">)</span> |
| |
| |
| <span class="n">Generator</span> <span class="o">=</span> <span class="n">_Random</span> |
| |
| |
| <div class="viewcode-block" id="parse_byte_size"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.parse_byte_size">[docs]</a><span class="k">def</span> <span class="nf">parse_byte_size</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="n">suffixes</span> <span class="o">=</span> <span class="s1">'BKMGTP'</span> |
| <span class="k">if</span> <span class="n">s</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> <span class="ow">in</span> <span class="n">suffixes</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">int</span><span class="p">(</span><span class="nb">float</span><span class="p">(</span><span class="n">s</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">])</span> <span class="o">*</span> <span class="mi">1024</span><span class="o">**</span><span class="n">suffixes</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">s</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="nb">int</span><span class="p">(</span><span class="n">s</span><span class="p">)</span></div> |
| |
| |
| <div class="viewcode-block" id="div_round_up"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.div_round_up">[docs]</a><span class="k">def</span> <span class="nf">div_round_up</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Return ceil(a/b)."""</span> |
| <span class="k">return</span> <span class="nb">int</span><span class="p">(</span><span class="n">math</span><span class="o">.</span><span class="n">ceil</span><span class="p">(</span><span class="nb">float</span><span class="p">(</span><span class="n">a</span><span class="p">)</span> <span class="o">/</span> <span class="n">b</span><span class="p">))</span></div> |
| |
| |
| <div class="viewcode-block" id="rotate_key"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.rotate_key">[docs]</a><span class="k">def</span> <span class="nf">rotate_key</span><span class="p">(</span><span class="n">element</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns a new key-value pair of the same size but with a different key."""</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="o">=</span> <span class="n">element</span> |
| <span class="k">return</span> <span class="n">key</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">:]</span> <span class="o">+</span> <span class="n">key</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">],</span> <span class="n">value</span></div> |
| |
| |
| <div class="viewcode-block" id="initial_splitting_zipf"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.initial_splitting_zipf">[docs]</a><span class="k">def</span> <span class="nf">initial_splitting_zipf</span><span class="p">(</span> |
| <span class="n">start_position</span><span class="p">,</span> |
| <span class="n">stop_position</span><span class="p">,</span> |
| <span class="n">desired_num_bundles</span><span class="p">,</span> |
| <span class="n">distribution_parameter</span><span class="p">,</span> |
| <span class="n">num_total_records</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Split the given range (defined by start_position, stop_position) into</span> |
| <span class="sd"> desired_num_bundles using zipf with the given distribution_parameter.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">num_total_records</span><span class="p">:</span> |
| <span class="n">num_total_records</span> <span class="o">=</span> <span class="n">stop_position</span> <span class="o">-</span> <span class="n">start_position</span> |
| <span class="n">samples</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">random</span><span class="o">.</span><span class="n">zipf</span><span class="p">(</span><span class="n">distribution_parameter</span><span class="p">,</span> <span class="n">desired_num_bundles</span><span class="p">)</span> |
| <span class="n">total</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">samples</span><span class="p">)</span> |
| <span class="n">relative_bundle_sizes</span> <span class="o">=</span> <span class="p">[(</span><span class="nb">float</span><span class="p">(</span><span class="n">sample</span><span class="p">)</span> <span class="o">/</span> <span class="n">total</span><span class="p">)</span> <span class="k">for</span> <span class="n">sample</span> <span class="ow">in</span> <span class="n">samples</span><span class="p">]</span> |
| <span class="n">bundle_ranges</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">start</span> <span class="o">=</span> <span class="n">start_position</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="k">while</span> <span class="n">start</span> <span class="o"><</span> <span class="n">stop_position</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">index</span> <span class="o">==</span> <span class="n">desired_num_bundles</span> <span class="o">-</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="n">bundle_ranges</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">start</span><span class="p">,</span> <span class="n">stop_position</span><span class="p">))</span> |
| <span class="k">break</span> |
| <span class="n">stop</span> <span class="o">=</span> <span class="n">start</span> <span class="o">+</span> <span class="nb">int</span><span class="p">(</span><span class="n">num_total_records</span> <span class="o">*</span> <span class="n">relative_bundle_sizes</span><span class="p">[</span><span class="n">index</span><span class="p">])</span> |
| <span class="n">bundle_ranges</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">start</span><span class="p">,</span> <span class="n">stop</span><span class="p">))</span> |
| <span class="n">start</span> <span class="o">=</span> <span class="n">stop</span> |
| <span class="n">index</span> <span class="o">+=</span> <span class="mi">1</span> |
| <span class="k">return</span> <span class="n">bundle_ranges</span></div> |
| |
| |
| <div class="viewcode-block" id="SyntheticStep"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticStep">[docs]</a><span class="k">class</span> <span class="nc">SyntheticStep</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A DoFn of which behavior can be controlled through prespecified parameters.</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">per_element_delay_sec</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">per_bundle_delay_sec</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">output_records_per_input_record</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span> |
| <span class="n">output_filter_ratio</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">per_element_delay_sec</span> <span class="ow">and</span> <span class="n">per_element_delay_sec</span> <span class="o"><</span> <span class="mf">1e-3</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Per element sleep time must be at least 1e-3. '</span> |
| <span class="s1">'Received: </span><span class="si">%r</span><span class="s1">'</span><span class="p">,</span> |
| <span class="n">per_element_delay_sec</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_per_element_delay_sec</span> <span class="o">=</span> <span class="n">per_element_delay_sec</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_per_bundle_delay_sec</span> <span class="o">=</span> <span class="n">per_bundle_delay_sec</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_output_records_per_input_record</span> <span class="o">=</span> <span class="n">output_records_per_input_record</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_output_filter_ratio</span> <span class="o">=</span> <span class="n">output_filter_ratio</span> |
| |
| <div class="viewcode-block" id="SyntheticStep.start_bundle"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticStep.start_bundle">[docs]</a> <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">_start_time</span> <span class="o">=</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span></div> |
| |
| <div class="viewcode-block" id="SyntheticStep.finish_bundle"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticStep.finish_bundle">[docs]</a> <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="c1"># The target is for the enclosing stage to take as close to as possible</span> |
| <span class="c1"># the given number of seconds, so we only sleep enough to make up for</span> |
| <span class="c1"># overheads not incurred elsewhere.</span> |
| <span class="n">to_sleep</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_per_bundle_delay_sec</span> <span class="o">-</span> <span class="p">(</span><span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> <span class="o">-</span> <span class="bp">self</span><span class="o">.</span><span class="n">_start_time</span><span class="p">)</span> |
| |
| <span class="c1"># Ignoring sub-millisecond sleep times.</span> |
| <span class="k">if</span> <span class="n">to_sleep</span> <span class="o">>=</span> <span class="mf">1e-3</span><span class="p">:</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="n">to_sleep</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="SyntheticStep.process"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticStep.process">[docs]</a> <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="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_per_element_delay_sec</span> <span class="o">>=</span> <span class="mf">1e-3</span><span class="p">:</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_per_element_delay_sec</span><span class="p">)</span> |
| <span class="n">filter_element</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_output_filter_ratio</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">np</span><span class="o">.</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="bp">self</span><span class="o">.</span><span class="n">_output_filter_ratio</span><span class="p">:</span> |
| <span class="n">filter_element</span> <span class="o">=</span> <span class="kc">True</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">filter_element</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_output_records_per_input_record</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">element</span></div></div> |
| |
| |
| <div class="viewcode-block" id="NonLiquidShardingOffsetRangeTracker"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.NonLiquidShardingOffsetRangeTracker">[docs]</a><span class="k">class</span> <span class="nc">NonLiquidShardingOffsetRangeTracker</span><span class="p">(</span><span class="n">OffsetRestrictionTracker</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""An OffsetRangeTracker that doesn't allow splitting. """</span> |
| <div class="viewcode-block" id="NonLiquidShardingOffsetRangeTracker.try_split"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.NonLiquidShardingOffsetRangeTracker.try_split">[docs]</a> <span class="k">def</span> <span class="nf">try_split</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">split_offset</span><span class="p">):</span> |
| <span class="k">pass</span> <span class="c1"># Don't split.</span></div> |
| |
| <div class="viewcode-block" id="NonLiquidShardingOffsetRangeTracker.checkpoint"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.NonLiquidShardingOffsetRangeTracker.checkpoint">[docs]</a> <span class="k">def</span> <span class="nf">checkpoint</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">pass</span> <span class="c1"># Don't split.</span></div></div> |
| |
| |
| <div class="viewcode-block" id="SyntheticSDFStepRestrictionProvider"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFStepRestrictionProvider">[docs]</a><span class="k">class</span> <span class="nc">SyntheticSDFStepRestrictionProvider</span><span class="p">(</span><span class="n">RestrictionProvider</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A `RestrictionProvider` for SyntheticSDFStep.</span> |
| |
| <span class="sd"> An initial_restriction and split that operate on num_records and ignores</span> |
| <span class="sd"> source description (element). Splits into initial_splitting_num_bundles.</span> |
| <span class="sd"> Returns size_estimate_override as restriction size, if set. Otherwise uses</span> |
| <span class="sd"> element size.</span> |
| |
| <span class="sd"> If initial_splitting_uneven_chunks, produces uneven chunks.</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">num_records</span><span class="p">,</span> |
| <span class="n">initial_splitting_num_bundles</span><span class="p">,</span> |
| <span class="n">initial_splitting_uneven_chunks</span><span class="p">,</span> |
| <span class="n">disable_liquid_sharding</span><span class="p">,</span> |
| <span class="n">size_estimate_override</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_num_records</span> <span class="o">=</span> <span class="n">num_records</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_num_bundles</span> <span class="o">=</span> <span class="n">initial_splitting_num_bundles</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_uneven_chunks</span> <span class="o">=</span> <span class="n">initial_splitting_uneven_chunks</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_disable_liquid_sharding</span> <span class="o">=</span> <span class="n">disable_liquid_sharding</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_size_estimate_override</span> <span class="o">=</span> <span class="n">size_estimate_override</span> |
| |
| <div class="viewcode-block" id="SyntheticSDFStepRestrictionProvider.initial_restriction"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFStepRestrictionProvider.initial_restriction">[docs]</a> <span class="k">def</span> <span class="nf">initial_restriction</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="k">return</span> <span class="n">OffsetRange</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_records</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSDFStepRestrictionProvider.create_tracker"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFStepRestrictionProvider.create_tracker">[docs]</a> <span class="k">def</span> <span class="nf">create_tracker</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_disable_liquid_sharding</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">NonLiquidShardingOffsetRangeTracker</span><span class="p">(</span><span class="n">restriction</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">OffsetRestrictionTracker</span><span class="p">(</span><span class="n">restriction</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSDFStepRestrictionProvider.split"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFStepRestrictionProvider.split">[docs]</a> <span class="k">def</span> <span class="nf">split</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">restriction</span><span class="p">):</span> |
| <span class="n">elems</span> <span class="o">=</span> <span class="n">restriction</span><span class="o">.</span><span class="n">size</span><span class="p">()</span> |
| <span class="k">if</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_uneven_chunks</span> <span class="ow">and</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_num_bundles</span> <span class="o">></span> <span class="mi">1</span> <span class="ow">and</span> <span class="n">elems</span> <span class="o">></span> <span class="mi">1</span><span class="p">):</span> |
| <span class="n">bundle_ranges</span> <span class="o">=</span> <span class="n">initial_splitting_zipf</span><span class="p">(</span> |
| <span class="n">restriction</span><span class="o">.</span><span class="n">start</span><span class="p">,</span> |
| <span class="n">restriction</span><span class="o">.</span><span class="n">stop</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_num_bundles</span><span class="p">,</span> |
| <span class="mf">3.0</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">start</span><span class="p">,</span> <span class="n">stop</span> <span class="ow">in</span> <span class="n">bundle_ranges</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">OffsetRange</span><span class="p">(</span><span class="n">start</span><span class="p">,</span> <span class="n">stop</span><span class="p">)</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">offsets_per_split</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="p">(</span><span class="n">elems</span> <span class="o">//</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_num_bundles</span><span class="p">))</span> |
| <span class="k">for</span> <span class="n">split</span> <span class="ow">in</span> <span class="n">restriction</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="n">offsets_per_split</span><span class="p">,</span> <span class="n">offsets_per_split</span> <span class="o">//</span> <span class="mi">2</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">split</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSDFStepRestrictionProvider.restriction_size"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFStepRestrictionProvider.restriction_size">[docs]</a> <span class="k">def</span> <span class="nf">restriction_size</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">restriction</span><span class="p">):</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_size_estimate_override</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_size_estimate_override</span> |
| <span class="n">element_size</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">element</span><span class="p">)</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">element</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="k">else</span> <span class="mi">1</span> |
| <span class="k">return</span> <span class="n">restriction</span><span class="o">.</span><span class="n">size</span><span class="p">()</span> <span class="o">*</span> <span class="n">element_size</span></div></div> |
| |
| |
| <div class="viewcode-block" id="get_synthetic_sdf_step"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.get_synthetic_sdf_step">[docs]</a><span class="k">def</span> <span class="nf">get_synthetic_sdf_step</span><span class="p">(</span> |
| <span class="n">per_element_delay_sec</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">per_bundle_delay_sec</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">output_records_per_input_record</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span> |
| <span class="n">output_filter_ratio</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">initial_splitting_num_bundles</span><span class="o">=</span><span class="mi">8</span><span class="p">,</span> |
| <span class="n">initial_splitting_uneven_chunks</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">disable_liquid_sharding</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">size_estimate_override</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A function which returns a SyntheticSDFStep with given parameters. """</span> |
| <span class="k">class</span> <span class="nc">SyntheticSDFStep</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A SplittableDoFn of which behavior can be controlled through prespecified</span> |
| <span class="sd"> parameters.</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">per_element_delay_sec_arg</span><span class="p">,</span> |
| <span class="n">per_bundle_delay_sec_arg</span><span class="p">,</span> |
| <span class="n">output_filter_ratio_arg</span><span class="p">,</span> |
| <span class="n">output_records_per_input_record_arg</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">per_element_delay_sec_arg</span><span class="p">:</span> |
| <span class="n">per_element_delay_sec_arg</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">per_element_delay_sec_arg</span> <span class="o">//</span> <span class="n">output_records_per_input_record_arg</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">per_element_delay_sec_arg</span> <span class="o"><</span> <span class="mf">1e-3</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Per element sleep time must be at least 1e-3 after being '</span> |
| <span class="s1">'divided among output elements.'</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_per_element_delay_sec</span> <span class="o">=</span> <span class="n">per_element_delay_sec_arg</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_per_bundle_delay_sec</span> <span class="o">=</span> <span class="n">per_bundle_delay_sec_arg</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_output_filter_ratio</span> <span class="o">=</span> <span class="n">output_filter_ratio_arg</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">_start_time</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="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># The target is for the enclosing stage to take as close to as possible</span> |
| <span class="c1"># the given number of seconds, so we only sleep enough to make up for</span> |
| <span class="c1"># overheads not incurred elsewhere.</span> |
| <span class="n">to_sleep</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_per_bundle_delay_sec</span> <span class="o">-</span> <span class="p">(</span><span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> <span class="o">-</span> <span class="bp">self</span><span class="o">.</span><span class="n">_start_time</span><span class="p">)</span> |
| |
| <span class="c1"># Ignoring sub-millisecond sleep times.</span> |
| <span class="k">if</span> <span class="n">to_sleep</span> <span class="o">>=</span> <span class="mf">1e-3</span><span class="p">:</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="n">to_sleep</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">restriction_tracker</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">RestrictionParam</span><span class="p">(</span> |
| <span class="n">SyntheticSDFStepRestrictionProvider</span><span class="p">(</span> |
| <span class="n">output_records_per_input_record</span><span class="p">,</span> |
| <span class="n">initial_splitting_num_bundles</span><span class="p">,</span> |
| <span class="n">initial_splitting_uneven_chunks</span><span class="p">,</span> |
| <span class="n">disable_liquid_sharding</span><span class="p">,</span> |
| <span class="n">size_estimate_override</span><span class="p">))):</span> |
| <span class="n">filter_element</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_output_filter_ratio</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">np</span><span class="o">.</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="bp">self</span><span class="o">.</span><span class="n">_output_filter_ratio</span><span class="p">:</span> |
| <span class="n">filter_element</span> <span class="o">=</span> <span class="kc">True</span> |
| |
| <span class="n">current_restriction</span> <span class="o">=</span> <span class="n">restriction_tracker</span><span class="o">.</span><span class="n">current_restriction</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">cur</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">current_restriction</span><span class="o">.</span><span class="n">start</span><span class="p">,</span> <span class="n">current_restriction</span><span class="o">.</span><span class="n">stop</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">restriction_tracker</span><span class="o">.</span><span class="n">try_claim</span><span class="p">(</span><span class="n">cur</span><span class="p">):</span> |
| <span class="k">return</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_per_element_delay_sec</span><span class="p">:</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_per_element_delay_sec</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">filter_element</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">element</span> |
| <span class="n">cur</span> <span class="o">+=</span> <span class="mi">1</span> |
| |
| <span class="k">return</span> <span class="n">SyntheticSDFStep</span><span class="p">(</span> |
| <span class="n">per_element_delay_sec</span><span class="p">,</span> |
| <span class="n">per_bundle_delay_sec</span><span class="p">,</span> |
| <span class="n">output_filter_ratio</span><span class="p">,</span> |
| <span class="n">output_records_per_input_record</span><span class="p">)</span></div> |
| |
| |
| <div class="viewcode-block" id="SyntheticSource"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSource">[docs]</a><span class="k">class</span> <span class="nc">SyntheticSource</span><span class="p">(</span><span class="n">iobase</span><span class="o">.</span><span class="n">BoundedSource</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A custom source of a specified size.</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">input_spec</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Initiates a synthetic source.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> input_spec: Input specification of the source. See corresponding option in</span> |
| <span class="sd"> function 'parse_args()' below for more details.</span> |
| <span class="sd"> Raises:</span> |
| <span class="sd"> ValueError: if input parameters are invalid.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="nf">maybe_parse_byte_size</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">parse_byte_size</span><span class="p">(</span><span class="n">s</span><span class="p">)</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="k">else</span> <span class="nb">int</span><span class="p">(</span><span class="n">s</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_num_records</span> <span class="o">=</span> <span class="n">input_spec</span><span class="p">[</span><span class="s1">'numRecords'</span><span class="p">]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_key_size</span> <span class="o">=</span> <span class="n">maybe_parse_byte_size</span><span class="p">(</span><span class="n">input_spec</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'keySizeBytes'</span><span class="p">,</span> <span class="mi">1</span><span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_hot_key_fraction</span> <span class="o">=</span> <span class="n">input_spec</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'hotKeyFraction'</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_hot_keys</span> <span class="o">=</span> <span class="n">input_spec</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'numHotKeys'</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">_value_size</span> <span class="o">=</span> <span class="n">maybe_parse_byte_size</span><span class="p">(</span> |
| <span class="n">input_spec</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'valueSizeBytes'</span><span class="p">,</span> <span class="mi">1</span><span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_total_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">element_size</span> <span class="o">*</span> <span class="bp">self</span><span class="o">.</span><span class="n">_num_records</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">input_spec</span><span class="p">[</span><span class="s1">'bundleSizeDistribution'</span><span class="p">][</span><span class="s1">'type'</span><span class="p">]</span> |
| <span class="k">if</span> <span class="s1">'bundleSizeDistribution'</span> <span class="ow">in</span> <span class="n">input_spec</span> <span class="k">else</span> <span class="s1">'const'</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting</span> <span class="o">!=</span> <span class="s1">'const'</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting</span> <span class="o">!=</span> <span class="s1">'zipf'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Only const and zipf distributions are supported for determining '</span> |
| <span class="s1">'sizes of bundles produced by initial splitting. Received: </span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_num_bundles</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">input_spec</span><span class="p">[</span><span class="s1">'forceNumInitialBundles'</span><span class="p">]</span> |
| <span class="k">if</span> <span class="s1">'forceNumInitialBundles'</span> <span class="ow">in</span> <span class="n">input_spec</span> <span class="k">else</span> <span class="mi">0</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting</span> <span class="o">==</span> <span class="s1">'zipf'</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_distribution_parameter</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">input_spec</span><span class="p">[</span><span class="s1">'bundleSizeDistribution'</span><span class="p">][</span><span class="s1">'param'</span><span class="p">])</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_distribution_parameter</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="s1">'Parameter for a Zipf distribution must be larger than 1. '</span> |
| <span class="s1">'Received </span><span class="si">%r</span><span class="s1">.'</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_distribution_parameter</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">_initial_splitting_distribution_parameter</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_dynamic_splitting</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s1">'none'</span> <span class="k">if</span> <span class="p">(</span> |
| <span class="s1">'splitPointFrequencyRecords'</span> <span class="ow">in</span> <span class="n">input_spec</span> <span class="ow">and</span> |
| <span class="n">input_spec</span><span class="p">[</span><span class="s1">'splitPointFrequencyRecords'</span><span class="p">]</span> <span class="o">==</span> <span class="mi">0</span><span class="p">)</span> <span class="k">else</span> <span class="s1">'perfect'</span><span class="p">)</span> |
| <span class="k">if</span> <span class="s1">'delayDistribution'</span> <span class="ow">in</span> <span class="n">input_spec</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">input_spec</span><span class="p">[</span><span class="s1">'delayDistribution'</span><span class="p">][</span><span class="s1">'type'</span><span class="p">]</span> <span class="o">!=</span> <span class="s1">'const'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'SyntheticSource currently only supports delay '</span> |
| <span class="s1">'distributions of type </span><span class="se">\'</span><span class="s1">const</span><span class="se">\'</span><span class="s1">. Received </span><span class="si">%s</span><span class="s1">.'</span><span class="p">,</span> |
| <span class="n">input_spec</span><span class="p">[</span><span class="s1">'delayDistribution'</span><span class="p">][</span><span class="s1">'type'</span><span class="p">])</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_sleep_per_input_record_sec</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">float</span><span class="p">(</span><span class="n">input_spec</span><span class="p">[</span><span class="s1">'delayDistribution'</span><span class="p">][</span><span class="s1">'const'</span><span class="p">])</span> <span class="o">/</span> <span class="mi">1000</span><span class="p">)</span> |
| <span class="k">if</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_sleep_per_input_record_sec</span> <span class="ow">and</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_sleep_per_input_record_sec</span> <span class="o"><</span> <span class="mf">1e-3</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Sleep time per input record must be at least 1e-3.'</span> |
| <span class="s1">' Received: </span><span class="si">%r</span><span class="s1">'</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_sleep_per_input_record_sec</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">_sleep_per_input_record_sec</span> <span class="o">=</span> <span class="mi">0</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">element_size</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">_key_size</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_value_size</span> |
| |
| <div class="viewcode-block" id="SyntheticSource.estimate_size"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSource.estimate_size">[docs]</a> <span class="k">def</span> <span class="nf">estimate_size</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">_total_size</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSource.split"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSource.split">[docs]</a> <span class="k">def</span> <span class="nf">split</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">desired_bundle_size</span><span class="p">,</span> <span class="n">start_position</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">stop_position</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="c1"># Performs initial splitting of SyntheticSource.</span> |
| <span class="c1">#</span> |
| <span class="c1"># Exact sizes and distribution of initial splits generated here depends on</span> |
| <span class="c1"># the input specification of the SyntheticSource.</span> |
| |
| <span class="k">if</span> <span class="n">stop_position</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">stop_position</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_num_records</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting</span> <span class="o">==</span> <span class="s1">'zipf'</span><span class="p">:</span> |
| <span class="n">desired_num_bundles</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_num_bundles</span> <span class="ow">or</span> <span class="n">math</span><span class="o">.</span><span class="n">ceil</span><span class="p">(</span> |
| <span class="nb">float</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">estimate_size</span><span class="p">())</span> <span class="o">/</span> <span class="n">desired_bundle_size</span><span class="p">)</span> |
| <span class="n">bundle_ranges</span> <span class="o">=</span> <span class="n">initial_splitting_zipf</span><span class="p">(</span> |
| <span class="n">start_position</span><span class="p">,</span> |
| <span class="n">stop_position</span><span class="p">,</span> |
| <span class="n">desired_num_bundles</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_distribution_parameter</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_num_records</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_num_bundles</span><span class="p">:</span> |
| <span class="n">bundle_size_in_elements</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span> |
| <span class="mi">1</span><span class="p">,</span> <span class="nb">int</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_num_records</span> <span class="o">/</span> <span class="bp">self</span><span class="o">.</span><span class="n">_initial_splitting_num_bundles</span><span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">bundle_size_in_elements</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">max</span><span class="p">(</span> |
| <span class="n">div_round_up</span><span class="p">(</span><span class="n">desired_bundle_size</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">element_size</span><span class="p">),</span> |
| <span class="nb">int</span><span class="p">(</span><span class="n">math</span><span class="o">.</span><span class="n">floor</span><span class="p">(</span><span class="n">math</span><span class="o">.</span><span class="n">sqrt</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_num_records</span><span class="p">)))))</span> |
| <span class="n">bundle_ranges</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">start</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">start_position</span><span class="p">,</span> <span class="n">stop_position</span><span class="p">,</span> |
| <span class="n">bundle_size_in_elements</span><span class="p">):</span> |
| <span class="n">stop</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">start</span> <span class="o">+</span> <span class="n">bundle_size_in_elements</span><span class="p">,</span> <span class="n">stop_position</span><span class="p">)</span> |
| <span class="n">bundle_ranges</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">start</span><span class="p">,</span> <span class="n">stop</span><span class="p">))</span> |
| |
| <span class="k">for</span> <span class="n">start</span><span class="p">,</span> <span class="n">stop</span> <span class="ow">in</span> <span class="n">bundle_ranges</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">iobase</span><span class="o">.</span><span class="n">SourceBundle</span><span class="p">(</span><span class="n">stop</span> <span class="o">-</span> <span class="n">start</span><span class="p">,</span> <span class="bp">self</span><span class="p">,</span> <span class="n">start</span><span class="p">,</span> <span class="n">stop</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSource.get_range_tracker"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSource.get_range_tracker">[docs]</a> <span class="k">def</span> <span class="nf">get_range_tracker</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">start_position</span><span class="p">,</span> <span class="n">stop_position</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">start_position</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">start_position</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="k">if</span> <span class="n">stop_position</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">stop_position</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_num_records</span> |
| <span class="n">tracker</span> <span class="o">=</span> <span class="n">range_trackers</span><span class="o">.</span><span class="n">OffsetRangeTracker</span><span class="p">(</span><span class="n">start_position</span><span class="p">,</span> <span class="n">stop_position</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_dynamic_splitting</span> <span class="o">==</span> <span class="s1">'none'</span><span class="p">:</span> |
| <span class="n">tracker</span> <span class="o">=</span> <span class="n">range_trackers</span><span class="o">.</span><span class="n">UnsplittableRangeTracker</span><span class="p">(</span><span class="n">tracker</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">tracker</span></div> |
| |
| <span class="k">def</span> <span class="nf">_gen_kv_pair</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">generator</span><span class="p">,</span> <span class="n">index</span><span class="p">):</span> |
| <span class="n">generator</span><span class="o">.</span><span class="n">seed</span><span class="p">(</span><span class="n">index</span><span class="p">)</span> |
| <span class="n">rand</span> <span class="o">=</span> <span class="n">generator</span><span class="o">.</span><span class="n">random_sample</span><span class="p">()</span> |
| |
| <span class="c1"># Determines whether to generate hot key or not.</span> |
| <span class="k">if</span> <span class="n">rand</span> <span class="o"><</span> <span class="bp">self</span><span class="o">.</span><span class="n">_hot_key_fraction</span><span class="p">:</span> |
| <span class="c1"># Generate hot key.</span> |
| <span class="c1"># An integer is randomly selected from the range [0, numHotKeys-1]</span> |
| <span class="c1"># with equal probability.</span> |
| <span class="n">generator_hot</span> <span class="o">=</span> <span class="n">Generator</span><span class="p">(</span><span class="n">index</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_num_hot_keys</span><span class="p">)</span> |
| <span class="n">bytes_</span> <span class="o">=</span> <span class="n">generator_hot</span><span class="o">.</span><span class="n">bytes</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_key_size</span><span class="p">),</span> <span class="n">generator</span><span class="o">.</span><span class="n">bytes</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_value_size</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">bytes_</span> <span class="o">=</span> <span class="n">generator</span><span class="o">.</span><span class="n">bytes</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">element_size</span><span class="p">)</span> |
| <span class="n">bytes_</span> <span class="o">=</span> <span class="n">bytes_</span><span class="p">[:</span><span class="bp">self</span><span class="o">.</span><span class="n">_key_size</span><span class="p">],</span> <span class="n">bytes_</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_key_size</span><span class="p">:]</span> |
| <span class="k">return</span> <span class="n">bytes_</span> |
| |
| <div class="viewcode-block" id="SyntheticSource.read"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSource.read">[docs]</a> <span class="k">def</span> <span class="nf">read</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">range_tracker</span><span class="p">):</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="n">range_tracker</span><span class="o">.</span><span class="n">start_position</span><span class="p">()</span> |
| <span class="n">generator</span> <span class="o">=</span> <span class="n">Generator</span><span class="p">()</span> |
| <span class="k">while</span> <span class="n">range_tracker</span><span class="o">.</span><span class="n">try_claim</span><span class="p">(</span><span class="n">index</span><span class="p">):</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_sleep_per_input_record_sec</span><span class="p">)</span> |
| <span class="k">yield</span> <span class="bp">self</span><span class="o">.</span><span class="n">_gen_kv_pair</span><span class="p">(</span><span class="n">generator</span><span class="p">,</span> <span class="n">index</span><span class="p">)</span> |
| <span class="n">index</span> <span class="o">+=</span> <span class="mi">1</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSource.default_output_coder"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSource.default_output_coder">[docs]</a> <span class="k">def</span> <span class="nf">default_output_coder</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">TupleCoder</span><span class="p">(</span> |
| <span class="p">[</span><span class="n">beam</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">BytesCoder</span><span class="p">(),</span> <span class="n">beam</span><span class="o">.</span><span class="n">coders</span><span class="o">.</span><span class="n">BytesCoder</span><span class="p">()])</span></div></div> |
| |
| |
| <div class="viewcode-block" id="SyntheticSDFSourceRestrictionProvider"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFSourceRestrictionProvider">[docs]</a><span class="k">class</span> <span class="nc">SyntheticSDFSourceRestrictionProvider</span><span class="p">(</span><span class="n">RestrictionProvider</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A `RestrictionProvider` for SyntheticSDFAsSource.</span> |
| |
| <span class="sd"> In initial_restriction(element) and split(element), element means source</span> |
| <span class="sd"> description.</span> |
| <span class="sd"> A typical element is like:</span> |
| |
| <span class="sd"> {</span> |
| <span class="sd"> 'key_size': 1,</span> |
| <span class="sd"> 'value_size': 1,</span> |
| <span class="sd"> 'initial_splitting_num_bundles': 8,</span> |
| <span class="sd"> 'initial_splitting_desired_bundle_size': 2,</span> |
| <span class="sd"> 'sleep_per_input_record_sec': 0,</span> |
| <span class="sd"> 'initial_splitting' : 'const'</span> |
| |
| <span class="sd"> }</span> |
| |
| <span class="sd"> """</span> |
| <div class="viewcode-block" id="SyntheticSDFSourceRestrictionProvider.initial_restriction"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFSourceRestrictionProvider.initial_restriction">[docs]</a> <span class="k">def</span> <span class="nf">initial_restriction</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="k">return</span> <span class="n">OffsetRange</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">element</span><span class="p">[</span><span class="s1">'num_records'</span><span class="p">])</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSDFSourceRestrictionProvider.create_tracker"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFSourceRestrictionProvider.create_tracker">[docs]</a> <span class="k">def</span> <span class="nf">create_tracker</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">restriction</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">restriction_trackers</span><span class="o">.</span><span class="n">OffsetRestrictionTracker</span><span class="p">(</span><span class="n">restriction</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSDFSourceRestrictionProvider.split"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFSourceRestrictionProvider.split">[docs]</a> <span class="k">def</span> <span class="nf">split</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">restriction</span><span class="p">):</span> |
| <span class="n">bundle_ranges</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">start_position</span> <span class="o">=</span> <span class="n">restriction</span><span class="o">.</span><span class="n">start</span> |
| <span class="n">stop_position</span> <span class="o">=</span> <span class="n">restriction</span><span class="o">.</span><span class="n">stop</span> |
| <span class="n">element_size</span> <span class="o">=</span> <span class="n">element</span><span class="p">[</span><span class="s1">'key_size'</span><span class="p">]</span> <span class="o">+</span> <span class="n">element</span><span class="p">[</span><span class="s1">'value_size'</span><span class="p">]</span> |
| <span class="n">estimate_size</span> <span class="o">=</span> <span class="n">element_size</span> <span class="o">*</span> <span class="n">element</span><span class="p">[</span><span class="s1">'num_records'</span><span class="p">]</span> |
| <span class="k">if</span> <span class="n">element</span><span class="p">[</span><span class="s1">'initial_splitting'</span><span class="p">]</span> <span class="o">==</span> <span class="s1">'zipf'</span><span class="p">:</span> |
| <span class="n">desired_num_bundles</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">element</span><span class="p">[</span><span class="s1">'initial_splitting_num_bundles'</span><span class="p">]</span> <span class="ow">or</span> <span class="n">div_round_up</span><span class="p">(</span> |
| <span class="n">estimate_size</span><span class="p">,</span> <span class="n">element</span><span class="p">[</span><span class="s1">'initial_splitting_desired_bundle_size'</span><span class="p">]))</span> |
| <span class="n">samples</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">random</span><span class="o">.</span><span class="n">zipf</span><span class="p">(</span> |
| <span class="n">element</span><span class="p">[</span><span class="s1">'initial_splitting_distribution_parameter'</span><span class="p">],</span> |
| <span class="n">desired_num_bundles</span><span class="p">)</span> |
| <span class="n">total</span> <span class="o">=</span> <span class="nb">sum</span><span class="p">(</span><span class="n">samples</span><span class="p">)</span> |
| <span class="n">relative_bundle_sizes</span> <span class="o">=</span> <span class="p">[(</span><span class="nb">float</span><span class="p">(</span><span class="n">sample</span><span class="p">)</span> <span class="o">/</span> <span class="n">total</span><span class="p">)</span> <span class="k">for</span> <span class="n">sample</span> <span class="ow">in</span> <span class="n">samples</span><span class="p">]</span> |
| <span class="n">start</span> <span class="o">=</span> <span class="n">start_position</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="k">while</span> <span class="n">start</span> <span class="o"><</span> <span class="n">stop_position</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">index</span> <span class="o">==</span> <span class="n">desired_num_bundles</span> <span class="o">-</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="n">bundle_ranges</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">OffsetRange</span><span class="p">(</span><span class="n">start</span><span class="p">,</span> <span class="n">stop_position</span><span class="p">))</span> |
| <span class="k">break</span> |
| <span class="n">stop</span> <span class="o">=</span> <span class="n">start</span> <span class="o">+</span> <span class="nb">int</span><span class="p">(</span> |
| <span class="n">element</span><span class="p">[</span><span class="s1">'num_records'</span><span class="p">]</span> <span class="o">*</span> <span class="n">relative_bundle_sizes</span><span class="p">[</span><span class="n">index</span><span class="p">])</span> |
| <span class="n">bundle_ranges</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">OffsetRange</span><span class="p">(</span><span class="n">start</span><span class="p">,</span> <span class="n">stop</span><span class="p">))</span> |
| <span class="n">start</span> <span class="o">=</span> <span class="n">stop</span> |
| <span class="n">index</span> <span class="o">+=</span> <span class="mi">1</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">element</span><span class="p">[</span><span class="s1">'initial_splitting_num_bundles'</span><span class="p">]:</span> |
| <span class="n">bundle_size_in_elements</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span> |
| <span class="mi">1</span><span class="p">,</span> |
| <span class="nb">int</span><span class="p">(</span> |
| <span class="n">element</span><span class="p">[</span><span class="s1">'num_records'</span><span class="p">]</span> <span class="o">/</span> |
| <span class="n">element</span><span class="p">[</span><span class="s1">'initial_splitting_num_bundles'</span><span class="p">]))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">bundle_size_in_elements</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">max</span><span class="p">(</span> |
| <span class="n">div_round_up</span><span class="p">(</span> |
| <span class="n">element</span><span class="p">[</span><span class="s1">'initial_splitting_desired_bundle_size'</span><span class="p">],</span> |
| <span class="n">element_size</span><span class="p">),</span> |
| <span class="nb">int</span><span class="p">(</span><span class="n">math</span><span class="o">.</span><span class="n">floor</span><span class="p">(</span><span class="n">math</span><span class="o">.</span><span class="n">sqrt</span><span class="p">(</span><span class="n">element</span><span class="p">[</span><span class="s1">'num_records'</span><span class="p">])))))</span> |
| <span class="k">for</span> <span class="n">start</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">start_position</span><span class="p">,</span> <span class="n">stop_position</span><span class="p">,</span> |
| <span class="n">bundle_size_in_elements</span><span class="p">):</span> |
| <span class="n">stop</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">start</span> <span class="o">+</span> <span class="n">bundle_size_in_elements</span><span class="p">,</span> <span class="n">stop_position</span><span class="p">)</span> |
| <span class="n">bundle_ranges</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">OffsetRange</span><span class="p">(</span><span class="n">start</span><span class="p">,</span> <span class="n">stop</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">bundle_ranges</span></div> |
| |
| <div class="viewcode-block" id="SyntheticSDFSourceRestrictionProvider.restriction_size"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFSourceRestrictionProvider.restriction_size">[docs]</a> <span class="k">def</span> <span class="nf">restriction_size</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">restriction</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">(</span><span class="n">element</span><span class="p">[</span><span class="s1">'key_size'</span><span class="p">]</span> <span class="o">+</span> <span class="n">element</span><span class="p">[</span><span class="s1">'value_size'</span><span class="p">])</span> <span class="o">*</span> <span class="n">restriction</span><span class="o">.</span><span class="n">size</span><span class="p">()</span></div></div> |
| |
| |
| <div class="viewcode-block" id="SyntheticSDFAsSource"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFAsSource">[docs]</a><span class="k">class</span> <span class="nc">SyntheticSDFAsSource</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A SDF that generates records like a source.</span> |
| |
| <span class="sd"> This SDF accepts a PCollection of record-based source description.</span> |
| <span class="sd"> A typical description is like:</span> |
| |
| <span class="sd"> {</span> |
| <span class="sd"> 'key_size': 1,</span> |
| <span class="sd"> 'value_size': 1,</span> |
| <span class="sd"> 'initial_splitting_num_bundles': 8,</span> |
| <span class="sd"> 'initial_splitting_desired_bundle_size': 2,</span> |
| <span class="sd"> 'sleep_per_input_record_sec': 0,</span> |
| <span class="sd"> 'initial_splitting' : 'const'</span> |
| |
| <span class="sd"> }</span> |
| |
| <span class="sd"> A simple pipeline taking this SDF as a source is like:</span> |
| <span class="sd"> p</span> |
| <span class="sd"> | beam.Create([description1, description2,...])</span> |
| <span class="sd"> | beam.ParDo(SyntheticSDFAsSource())</span> |
| |
| <span class="sd"> NOTE:</span> |
| <span class="sd"> The SDF.process() will have different param content between defining a DoFn</span> |
| <span class="sd"> and runtime.</span> |
| <span class="sd"> When defining an SDF.process, the restriction_tracker should be a</span> |
| <span class="sd"> `RestrictionProvider`.</span> |
| <span class="sd"> During runtime, the DoFnRunner.process_with_sized_restriction() will feed</span> |
| <span class="sd"> a 'RestrictionTracker' based on a restriction to SDF.process().</span> |
| <span class="sd"> """</span> |
| <div class="viewcode-block" id="SyntheticSDFAsSource.process"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SyntheticSDFAsSource.process">[docs]</a> <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">restriction_tracker</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">RestrictionParam</span><span class="p">(</span> |
| <span class="n">SyntheticSDFSourceRestrictionProvider</span><span class="p">())):</span> |
| <span class="n">cur</span> <span class="o">=</span> <span class="n">restriction_tracker</span><span class="o">.</span><span class="n">current_restriction</span><span class="p">()</span><span class="o">.</span><span class="n">start</span> |
| <span class="k">while</span> <span class="n">restriction_tracker</span><span class="o">.</span><span class="n">try_claim</span><span class="p">(</span><span class="n">cur</span><span class="p">):</span> |
| <span class="n">r</span> <span class="o">=</span> <span class="n">Generator</span><span class="p">()</span> |
| <span class="n">r</span><span class="o">.</span><span class="n">seed</span><span class="p">(</span><span class="n">cur</span><span class="p">)</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="n">element</span><span class="p">[</span><span class="s1">'sleep_per_input_record_sec'</span><span class="p">])</span> |
| <span class="k">yield</span> <span class="n">r</span><span class="o">.</span><span class="n">bytes</span><span class="p">(</span><span class="n">element</span><span class="p">[</span><span class="s1">'key_size'</span><span class="p">]),</span> <span class="n">r</span><span class="o">.</span><span class="n">bytes</span><span class="p">(</span><span class="n">element</span><span class="p">[</span><span class="s1">'value_size'</span><span class="p">])</span> |
| <span class="n">cur</span> <span class="o">+=</span> <span class="mi">1</span></div></div> |
| |
| |
| <div class="viewcode-block" id="ShuffleBarrier"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.ShuffleBarrier">[docs]</a><span class="k">class</span> <span class="nc">ShuffleBarrier</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span> |
| <div class="viewcode-block" id="ShuffleBarrier.expand"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.ShuffleBarrier.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">pc</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">pc</span> |
| <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="n">rotate_key</span><span class="p">)</span> |
| <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">GroupByKey</span><span class="p">()</span> |
| <span class="o">|</span> <span class="s1">'Ungroup'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">FlatMap</span><span class="p">(</span><span class="k">lambda</span> <span class="n">elm</span><span class="p">:</span> <span class="p">[(</span><span class="n">elm</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">elm</span><span class="p">[</span><span class="mi">1</span><span class="p">]]))</span></div></div> |
| |
| |
| <div class="viewcode-block" id="SideInputBarrier"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SideInputBarrier">[docs]</a><span class="k">class</span> <span class="nc">SideInputBarrier</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span> |
| <div class="viewcode-block" id="SideInputBarrier.expand"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.SideInputBarrier.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">pc</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">pc</span> |
| <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="n">rotate_key</span><span class="p">)</span> |
| <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">elem</span><span class="p">,</span> |
| <span class="n">ignored</span><span class="p">:</span> <span class="n">elem</span><span class="p">,</span> |
| <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">AsIter</span><span class="p">(</span><span class="n">pc</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">FlatMap</span><span class="p">(</span><span class="k">lambda</span> <span class="n">elem</span><span class="p">:</span> <span class="kc">None</span><span class="p">))))</span></div></div> |
| |
| |
| <div class="viewcode-block" id="merge_using_gbk"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.merge_using_gbk">[docs]</a><span class="k">def</span> <span class="nf">merge_using_gbk</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">pc1</span><span class="p">,</span> <span class="n">pc2</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Merges two given PCollections using a CoGroupByKey."""</span> |
| |
| <span class="n">pc1_with_key</span> <span class="o">=</span> <span class="n">pc1</span> <span class="o">|</span> <span class="p">(</span><span class="n">name</span> <span class="o">+</span> <span class="s1">'AttachKey1'</span><span class="p">)</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">x</span><span class="p">))</span> |
| <span class="n">pc2_with_key</span> <span class="o">=</span> <span class="n">pc2</span> <span class="o">|</span> <span class="p">(</span><span class="n">name</span> <span class="o">+</span> <span class="s1">'AttachKey2'</span><span class="p">)</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">x</span><span class="p">))</span> |
| |
| <span class="n">grouped</span> <span class="o">=</span> <span class="p">({</span> |
| <span class="s1">'pc1'</span><span class="p">:</span> <span class="n">pc1_with_key</span><span class="p">,</span> <span class="s1">'pc2'</span><span class="p">:</span> <span class="n">pc2_with_key</span> |
| <span class="p">}</span> <span class="o">|</span> <span class="p">(</span><span class="n">name</span> <span class="o">+</span> <span class="s1">'Group'</span><span class="p">)</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">CoGroupByKey</span><span class="p">())</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">grouped</span> <span class="o">|</span> <span class="p">(</span><span class="n">name</span> <span class="o">+</span> <span class="s1">'DeDup'</span><span class="p">)</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">elm</span><span class="p">:</span> <span class="n">elm</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> |
| <span class="p">)</span> <span class="c1"># Ignoring values</span></div> |
| |
| |
| <div class="viewcode-block" id="merge_using_side_input"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.merge_using_side_input">[docs]</a><span class="k">def</span> <span class="nf">merge_using_side_input</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">pc1</span><span class="p">,</span> <span class="n">pc2</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Merges two given PCollections using side inputs."""</span> |
| <span class="k">def</span> <span class="nf">join_fn</span><span class="p">(</span><span class="n">val</span><span class="p">,</span> <span class="n">_</span><span class="p">):</span> <span class="c1"># Ignoring side input</span> |
| <span class="k">return</span> <span class="n">val</span> |
| |
| <span class="k">return</span> <span class="n">pc1</span> <span class="o">|</span> <span class="n">name</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="n">join_fn</span><span class="p">,</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">AsIter</span><span class="p">(</span><span class="n">pc2</span><span class="p">))</span></div> |
| |
| |
| <div class="viewcode-block" id="expand_using_gbk"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.expand_using_gbk">[docs]</a><span class="k">def</span> <span class="nf">expand_using_gbk</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">pc</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Expands a given PCollection into two copies using GroupByKey."""</span> |
| |
| <span class="n">ret</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">ret</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">pc</span> <span class="o">|</span> <span class="p">(</span><span class="s1">'</span><span class="si">%s</span><span class="s1">.a'</span> <span class="o">%</span> <span class="n">name</span><span class="p">)</span> <span class="o">>></span> <span class="n">ShuffleBarrier</span><span class="p">()))</span> |
| <span class="n">ret</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">pc</span> <span class="o">|</span> <span class="p">(</span><span class="s1">'</span><span class="si">%s</span><span class="s1">.b'</span> <span class="o">%</span> <span class="n">name</span><span class="p">)</span> <span class="o">>></span> <span class="n">ShuffleBarrier</span><span class="p">()))</span> |
| <span class="k">return</span> <span class="n">ret</span></div> |
| |
| |
| <div class="viewcode-block" id="expand_using_second_output"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.expand_using_second_output">[docs]</a><span class="k">def</span> <span class="nf">expand_using_second_output</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">pc</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Expands a given PCollection into two copies using side outputs."""</span> |
| <span class="k">class</span> <span class="nc">ExpandFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="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="k">yield</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">TaggedOutput</span><span class="p">(</span><span class="s1">'second_out'</span><span class="p">,</span> <span class="n">element</span><span class="p">)</span> |
| <span class="k">yield</span> <span class="n">element</span> |
| |
| <span class="n">pc1</span><span class="p">,</span> <span class="n">pc2</span> <span class="o">=</span> <span class="p">(</span><span class="n">pc</span> <span class="o">|</span> <span class="n">name</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span> |
| <span class="n">ExpandFn</span><span class="p">())</span><span class="o">.</span><span class="n">with_outputs</span><span class="p">(</span><span class="s1">'second_out'</span><span class="p">,</span> <span class="n">main</span><span class="o">=</span><span class="s1">'main_out'</span><span class="p">))</span> |
| <span class="k">return</span> <span class="p">[</span><span class="n">pc1</span><span class="p">,</span> <span class="n">pc2</span><span class="p">]</span></div> |
| |
| |
| <span class="k">def</span> <span class="nf">_parse_steps</span><span class="p">(</span><span class="n">json_str</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Converts the JSON step description into Python objects.</span> |
| |
| <span class="sd"> See property 'steps' for more details about the JSON step description.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> json_str: a JSON string that describes the steps.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> Information about steps as a list of dictionaries. Each dictionary may have</span> |
| <span class="sd"> following properties.</span> |
| <span class="sd"> (1) per_element_delay - amount of delay for each element in seconds.</span> |
| <span class="sd"> (2) per_bundle_delay - minimum amount of delay for a given step in seconds.</span> |
| <span class="sd"> (3) output_records_per_input_record - number of output elements generated</span> |
| <span class="sd"> for each input element to a step.</span> |
| <span class="sd"> (4) output_filter_ratio - the probability at which a step may filter out a</span> |
| <span class="sd"> given element by not producing any output for that element.</span> |
| <span class="sd"> (5) splittable - if the step should be splittable.</span> |
| <span class="sd"> (6) initial_splitting_num_bundles - number of bundles initial split if step</span> |
| <span class="sd"> is splittable.</span> |
| <span class="sd"> (7) initial_splitting_uneven_chunks - if the bundles should be</span> |
| <span class="sd"> unevenly-sized</span> |
| <span class="sd"> (8) disable_liquid_sharding - if liquid sharding should be disabled</span> |
| <span class="sd"> (9) size_estimate_override - the size estimate or None to use default</span> |
| <span class="sd"> """</span> |
| <span class="n">all_steps</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">json_data</span> <span class="o">=</span> <span class="n">json</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span><span class="n">json_str</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">val</span> <span class="ow">in</span> <span class="n">json_data</span><span class="p">:</span> |
| <span class="n">steps</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'per_element_delay'</span><span class="p">]</span> <span class="o">=</span> <span class="p">((</span><span class="nb">float</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'per_element_delay_msec'</span><span class="p">])</span> <span class="o">/</span> <span class="mi">1000</span><span class="p">)</span> |
| <span class="k">if</span> <span class="s1">'per_element_delay_msec'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="mi">0</span><span class="p">)</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'per_bundle_delay'</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">float</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'per_bundle_delay_sec'</span><span class="p">])</span> |
| <span class="k">if</span> <span class="s1">'per_bundle_delay_sec'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="mi">0</span><span class="p">)</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'output_records_per_input_record'</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">int</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'output_records_per_input_record'</span><span class="p">])</span> |
| <span class="k">if</span> <span class="s1">'output_records_per_input_record'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="mi">1</span><span class="p">)</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'output_filter_ratio'</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">float</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'output_filter_ratio'</span><span class="p">])</span> |
| <span class="k">if</span> <span class="s1">'output_filter_ratio'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="mi">0</span><span class="p">)</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'splittable'</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">bool</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'splittable'</span><span class="p">])</span> <span class="k">if</span> <span class="s1">'splittable'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="kc">False</span><span class="p">)</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'initial_splitting_num_bundles'</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">int</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'initial_splitting_num_bundles'</span><span class="p">])</span> |
| <span class="k">if</span> <span class="s1">'initial_splitting_num_bundles'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="mi">8</span><span class="p">)</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'initial_splitting_uneven_chunks'</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">bool</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'initial_splitting_uneven_chunks'</span><span class="p">])</span> |
| <span class="k">if</span> <span class="s1">'initial_splitting_uneven_chunks'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="kc">False</span><span class="p">)</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'disable_liquid_sharding'</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">bool</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'disable_liquid_sharding'</span><span class="p">])</span> |
| <span class="k">if</span> <span class="s1">'disable_liquid_sharding'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="kc">False</span><span class="p">)</span> |
| <span class="n">steps</span><span class="p">[</span><span class="s1">'size_estimate_override'</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="nb">int</span><span class="p">(</span><span class="n">val</span><span class="p">[</span><span class="s1">'size_estimate_override'</span><span class="p">])</span> |
| <span class="k">if</span> <span class="s1">'size_estimate_override'</span> <span class="ow">in</span> <span class="n">val</span> <span class="k">else</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="n">all_steps</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">steps</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">all_steps</span> |
| |
| |
| <div class="viewcode-block" id="parse_args"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.parse_args">[docs]</a><span class="k">def</span> <span class="nf">parse_args</span><span class="p">(</span><span class="n">args</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Parses a given set of arguments.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> args: set of arguments to be passed.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> a tuple where first item gives the set of arguments defined and parsed</span> |
| <span class="sd"> within this method and second item gives the set of unknown arguments.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">parser</span> <span class="o">=</span> <span class="n">argparse</span><span class="o">.</span><span class="n">ArgumentParser</span><span class="p">()</span> |
| <span class="n">parser</span><span class="o">.</span><span class="n">add_argument</span><span class="p">(</span> |
| <span class="s1">'--steps'</span><span class="p">,</span> |
| <span class="n">dest</span><span class="o">=</span><span class="s1">'steps'</span><span class="p">,</span> |
| <span class="nb">type</span><span class="o">=</span><span class="n">_parse_steps</span><span class="p">,</span> |
| <span class="n">help</span><span class="o">=</span><span class="s1">'A JSON string that gives a list where each entry of the list is '</span> |
| <span class="s1">'configuration information for a step. Configuration for each step '</span> |
| <span class="s1">'consists of '</span> |
| <span class="s1">'(1) A float "per_bundle_delay_sec" (in seconds). Defaults to 0.'</span> |
| <span class="s1">'(2) A float "per_element_delay_msec" (in milli seconds). '</span> |
| <span class="s1">' Defaults to 0.'</span> |
| <span class="s1">'(3) An integer "output_records_per_input_record". Defaults to 1.'</span> |
| <span class="s1">'(4) A float "output_filter_ratio" in the range [0, 1] . '</span> |
| <span class="s1">' Defaults to 0.'</span> |
| <span class="s1">'(5) A bool "splittable" that defaults to false.'</span> |
| <span class="s1">'(6) An integer "initial_splitting_num_bundles". Defaults to 8.'</span><span class="p">)</span> |
| |
| <span class="n">parser</span><span class="o">.</span><span class="n">add_argument</span><span class="p">(</span> |
| <span class="s1">'--input'</span><span class="p">,</span> |
| <span class="n">dest</span><span class="o">=</span><span class="s1">'input'</span><span class="p">,</span> |
| <span class="nb">type</span><span class="o">=</span><span class="n">json</span><span class="o">.</span><span class="n">loads</span><span class="p">,</span> |
| <span class="n">help</span><span class="o">=</span><span class="s1">'A JSON string that describes the properties of the SyntheticSource '</span> |
| <span class="s1">'used by the pipeline. Configuration is similar to Java '</span> |
| <span class="s1">'SyntheticBoundedInput.'</span> |
| <span class="s1">'Currently supports following properties. '</span> |
| <span class="s1">'(1) An integer "numRecords". '</span> |
| <span class="s1">'(2) An integer "keySize". '</span> |
| <span class="s1">'(3) An integer "valueSize". '</span> |
| <span class="s1">'(4) A tuple "bundleSizeDistribution" with following values. '</span> |
| <span class="s1">' A string "type". Allowed values are "const" and "zipf". '</span> |
| <span class="s1">' An float "param". Only used if "type"=="zipf". Must be '</span> |
| <span class="s1">' larger than 1. '</span> |
| <span class="s1">'(5) An integer "forceNumInitialBundles". '</span> |
| <span class="s1">'(6) An integer "splitPointFrequencyRecords". '</span> |
| <span class="s1">'(7) A tuple "delayDistribution" with following values. '</span> |
| <span class="s1">' A string "type". Only allowed value is "const". '</span> |
| <span class="s1">' An integer "const". '</span><span class="p">)</span> |
| |
| <span class="n">parser</span><span class="o">.</span><span class="n">add_argument</span><span class="p">(</span> |
| <span class="s1">'--barrier'</span><span class="p">,</span> |
| <span class="n">dest</span><span class="o">=</span><span class="s1">'barrier'</span><span class="p">,</span> |
| <span class="n">default</span><span class="o">=</span><span class="s1">'shuffle'</span><span class="p">,</span> |
| <span class="n">choices</span><span class="o">=</span><span class="p">[</span> |
| <span class="s1">'shuffle'</span><span class="p">,</span> |
| <span class="s1">'side-input'</span><span class="p">,</span> |
| <span class="s1">'expand-gbk'</span><span class="p">,</span> |
| <span class="s1">'expand-second-output'</span><span class="p">,</span> |
| <span class="s1">'merge-gbk'</span><span class="p">,</span> |
| <span class="s1">'merge-side-input'</span> |
| <span class="p">],</span> |
| <span class="n">help</span><span class="o">=</span><span class="s1">'Whether to use shuffle as the barrier '</span> |
| <span class="s1">'(as opposed to side inputs).'</span><span class="p">)</span> |
| <span class="n">parser</span><span class="o">.</span><span class="n">add_argument</span><span class="p">(</span> |
| <span class="s1">'--output'</span><span class="p">,</span> |
| <span class="n">dest</span><span class="o">=</span><span class="s1">'output'</span><span class="p">,</span> |
| <span class="n">default</span><span class="o">=</span><span class="s1">''</span><span class="p">,</span> |
| <span class="n">help</span><span class="o">=</span><span class="s1">'Destination to write output.'</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">parser</span><span class="o">.</span><span class="n">parse_known_args</span><span class="p">(</span><span class="n">args</span><span class="p">)</span></div> |
| |
| |
| <div class="viewcode-block" id="run"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.run">[docs]</a><span class="k">def</span> <span class="nf">run</span><span class="p">(</span><span class="n">argv</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">save_main_session</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Runs the workflow."""</span> |
| <span class="n">known_args</span><span class="p">,</span> <span class="n">pipeline_args</span> <span class="o">=</span> <span class="n">parse_args</span><span class="p">(</span><span class="n">argv</span><span class="p">)</span> |
| |
| <span class="n">pipeline_options</span> <span class="o">=</span> <span class="n">PipelineOptions</span><span class="p">(</span><span class="n">pipeline_args</span><span class="p">)</span> |
| <span class="n">pipeline_options</span><span class="o">.</span><span class="n">view_as</span><span class="p">(</span><span class="n">SetupOptions</span><span class="p">)</span><span class="o">.</span><span class="n">save_main_session</span> <span class="o">=</span> <span class="n">save_main_session</span> |
| |
| <span class="n">input_info</span> <span class="o">=</span> <span class="n">known_args</span><span class="o">.</span><span class="n">input</span> |
| |
| <span class="k">with</span> <span class="n">TestPipeline</span><span class="p">(</span><span class="n">options</span><span class="o">=</span><span class="n">pipeline_options</span><span class="p">)</span> <span class="k">as</span> <span class="n">p</span><span class="p">:</span> |
| <span class="n">source</span> <span class="o">=</span> <span class="n">SyntheticSource</span><span class="p">(</span><span class="n">input_info</span><span class="p">)</span> |
| |
| <span class="c1"># pylint: disable=expression-not-assigned</span> |
| <span class="n">barrier</span> <span class="o">=</span> <span class="n">known_args</span><span class="o">.</span><span class="n">barrier</span> |
| |
| <span class="n">pc_list</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">num_roots</span> <span class="o">=</span> <span class="mi">2</span><span class="o">**</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">known_args</span><span class="o">.</span><span class="n">steps</span><span class="p">)</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)</span> <span class="k">if</span> <span class="p">(</span> |
| <span class="n">barrier</span> <span class="o">==</span> <span class="s1">'merge-gbk'</span> <span class="ow">or</span> <span class="n">barrier</span> <span class="o">==</span> <span class="s1">'merge-side-input'</span><span class="p">)</span> <span class="k">else</span> <span class="mi">1</span> |
| <span class="k">for</span> <span class="n">read_no</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">num_roots</span><span class="p">):</span> |
| <span class="n">pc_list</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">p</span> <span class="o">|</span> <span class="p">(</span><span class="s1">'Read </span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="n">read_no</span><span class="p">)</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">io</span><span class="o">.</span><span class="n">Read</span><span class="p">(</span><span class="n">source</span><span class="p">)))</span> |
| |
| <span class="k">for</span> <span class="n">step_no</span><span class="p">,</span> <span class="n">steps</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">known_args</span><span class="o">.</span><span class="n">steps</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">step_no</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">new_pc_list</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">pc_no</span><span class="p">,</span> <span class="n">pc</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">pc_list</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">barrier</span> <span class="o">==</span> <span class="s1">'shuffle'</span><span class="p">:</span> |
| <span class="n">new_pc_list</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="p">(</span><span class="n">pc</span> <span class="o">|</span> <span class="p">(</span><span class="s1">'shuffle </span><span class="si">%d</span><span class="s1">.</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">step_no</span><span class="p">,</span> <span class="n">pc_no</span><span class="p">))</span> <span class="o">>></span> <span class="n">ShuffleBarrier</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="n">barrier</span> <span class="o">==</span> <span class="s1">'side-input'</span><span class="p">:</span> |
| <span class="n">new_pc_list</span><span class="o">.</span><span class="n">append</span><span class="p">((</span> |
| <span class="n">pc</span> <span class="o">|</span> <span class="p">(</span><span class="s1">'side-input </span><span class="si">%d</span><span class="s1">.</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> |
| <span class="p">(</span><span class="n">step_no</span><span class="p">,</span> <span class="n">pc_no</span><span class="p">))</span> <span class="o">>></span> <span class="n">SideInputBarrier</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="n">barrier</span> <span class="o">==</span> <span class="s1">'expand-gbk'</span><span class="p">:</span> |
| <span class="n">new_pc_list</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span> |
| <span class="n">expand_using_gbk</span><span class="p">((</span><span class="s1">'expand-gbk </span><span class="si">%d</span><span class="s1">.</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">step_no</span><span class="p">,</span> <span class="n">pc_no</span><span class="p">)),</span> <span class="n">pc</span><span class="p">))</span> |
| <span class="k">elif</span> <span class="n">barrier</span> <span class="o">==</span> <span class="s1">'expand-second-output'</span><span class="p">:</span> |
| <span class="n">new_pc_list</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span> |
| <span class="n">expand_using_second_output</span><span class="p">(</span> |
| <span class="p">(</span><span class="s1">'expand-second-output </span><span class="si">%d</span><span class="s1">.</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">step_no</span><span class="p">,</span> <span class="n">pc_no</span><span class="p">)),</span> <span class="n">pc</span><span class="p">))</span> |
| <span class="k">elif</span> <span class="n">barrier</span> <span class="o">==</span> <span class="s1">'merge-gbk'</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">pc_no</span> <span class="o">%</span> <span class="mi">2</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">new_pc_list</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="n">merge_using_gbk</span><span class="p">((</span><span class="s1">'merge-gbk </span><span class="si">%d</span><span class="s1">.</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">step_no</span><span class="p">,</span> <span class="n">pc_no</span><span class="p">)),</span> |
| <span class="n">pc</span><span class="p">,</span> |
| <span class="n">pc_list</span><span class="p">[</span><span class="n">pc_no</span> <span class="o">+</span> <span class="mi">1</span><span class="p">]))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">continue</span> |
| <span class="k">elif</span> <span class="n">barrier</span> <span class="o">==</span> <span class="s1">'merge-side-input'</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">pc_no</span> <span class="o">%</span> <span class="mi">2</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">new_pc_list</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="n">merge_using_side_input</span><span class="p">(</span> |
| <span class="p">(</span><span class="s1">'merge-side-input </span><span class="si">%d</span><span class="s1">.</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">step_no</span><span class="p">,</span> <span class="n">pc_no</span><span class="p">)),</span> |
| <span class="n">pc</span><span class="p">,</span> |
| <span class="n">pc_list</span><span class="p">[</span><span class="n">pc_no</span> <span class="o">+</span> <span class="mi">1</span><span class="p">]))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">continue</span> |
| |
| <span class="n">pc_list</span> <span class="o">=</span> <span class="n">new_pc_list</span> |
| |
| <span class="n">new_pc_list</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">pc_no</span><span class="p">,</span> <span class="n">pc</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">pc_list</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">steps</span><span class="p">[</span><span class="s1">'splittable'</span><span class="p">]:</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="n">get_synthetic_sdf_step</span><span class="p">(</span> |
| <span class="n">per_element_delay_sec</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span><span class="s1">'per_element_delay'</span><span class="p">],</span> |
| <span class="n">per_bundle_delay_sec</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span><span class="s1">'per_bundle_delay'</span><span class="p">],</span> |
| <span class="n">output_records_per_input_record</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span> |
| <span class="s1">'output_records_per_input_record'</span><span class="p">],</span> |
| <span class="n">output_filter_ratio</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span><span class="s1">'output_filter_ratio'</span><span class="p">],</span> |
| <span class="n">initial_splitting_num_bundles</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span> |
| <span class="s1">'initial_splitting_num_bundles'</span><span class="p">],</span> |
| <span class="n">initial_splitting_uneven_chunks</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span> |
| <span class="s1">'initial_splitting_uneven_chunks'</span><span class="p">],</span> |
| <span class="n">disable_liquid_sharding</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span><span class="s1">'disable_liquid_sharding'</span><span class="p">],</span> |
| <span class="n">size_estimate_override</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span><span class="s1">'size_estimate_override'</span><span class="p">])</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">step</span> <span class="o">=</span> <span class="n">SyntheticStep</span><span class="p">(</span> |
| <span class="n">per_element_delay_sec</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span><span class="s1">'per_element_delay'</span><span class="p">],</span> |
| <span class="n">per_bundle_delay_sec</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span><span class="s1">'per_bundle_delay'</span><span class="p">],</span> |
| <span class="n">output_records_per_input_record</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span> |
| <span class="s1">'output_records_per_input_record'</span><span class="p">],</span> |
| <span class="n">output_filter_ratio</span><span class="o">=</span><span class="n">steps</span><span class="p">[</span><span class="s1">'output_filter_ratio'</span><span class="p">])</span> |
| <span class="n">new_pc</span> <span class="o">=</span> <span class="n">pc</span> <span class="o">|</span> <span class="s1">'SyntheticStep </span><span class="si">%d</span><span class="s1">.</span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">step_no</span><span class="p">,</span> |
| <span class="n">pc_no</span><span class="p">)</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="n">step</span><span class="p">)</span> |
| <span class="n">new_pc_list</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">new_pc</span><span class="p">)</span> |
| <span class="n">pc_list</span> <span class="o">=</span> <span class="n">new_pc_list</span> |
| |
| <span class="k">if</span> <span class="n">known_args</span><span class="o">.</span><span class="n">output</span><span class="p">:</span> |
| <span class="c1"># If an output location is provided we format and write output.</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pc_list</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="p">(</span> |
| <span class="n">pc_list</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> |
| <span class="o">|</span> <span class="s1">'FormatOutput'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">elm</span><span class="p">:</span> <span class="p">(</span><span class="n">elm</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">+</span> <span class="n">elm</span><span class="p">[</span><span class="mi">1</span><span class="p">]))</span> |
| <span class="o">|</span> <span class="s1">'WriteOutput'</span> <span class="o">>></span> <span class="n">WriteToText</span><span class="p">(</span><span class="n">known_args</span><span class="o">.</span><span class="n">output</span><span class="p">))</span> |
| |
| <span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">'Pipeline run completed.'</span><span class="p">)</span></div> |
| |
| |
| <span class="k">if</span> <span class="vm">__name__</span> <span class="o">==</span> <span class="s1">'__main__'</span><span class="p">:</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">getLogger</span><span class="p">()</span><span class="o">.</span><span class="n">setLevel</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="n">run</span><span class="p">()</span> |
| |
| |
| <div class="viewcode-block" id="StatefulLoadGenerator"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.StatefulLoadGenerator">[docs]</a><span class="k">class</span> <span class="nc">StatefulLoadGenerator</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A PTransform for generating random data using Timers API."""</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">input_options</span><span class="p">,</span> <span class="n">num_keys</span><span class="o">=</span><span class="mi">100</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">num_records</span> <span class="o">=</span> <span class="n">input_options</span><span class="p">[</span><span class="s1">'num_records'</span><span class="p">]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">key_size</span> <span class="o">=</span> <span class="n">input_options</span><span class="p">[</span><span class="s1">'key_size'</span><span class="p">]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">value_size</span> <span class="o">=</span> <span class="n">input_options</span><span class="p">[</span><span class="s1">'value_size'</span><span class="p">]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">num_keys</span> <span class="o">=</span> <span class="n">num_keys</span> |
| |
| <div class="viewcode-block" id="StatefulLoadGenerator.GenerateKeys"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.StatefulLoadGenerator.GenerateKeys">[docs]</a> <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">bytes</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">])</span> |
| <span class="k">class</span> <span class="nc">GenerateKeys</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="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_keys</span><span class="p">,</span> <span class="n">key_size</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">num_keys</span> <span class="o">=</span> <span class="n">num_keys</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">key_size</span> <span class="o">=</span> <span class="n">key_size</span> |
| |
| <div class="viewcode-block" id="StatefulLoadGenerator.GenerateKeys.process"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.StatefulLoadGenerator.GenerateKeys.process">[docs]</a> <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">impulse</span><span class="p">):</span> |
| <span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">num_keys</span><span class="p">):</span> |
| <span class="n">key</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">urandom</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">key_size</span><span class="p">)</span> |
| <span class="k">yield</span> <span class="n">key</span><span class="p">,</span> <span class="sa">b</span><span class="s1">''</span></div></div> |
| |
| <div class="viewcode-block" id="StatefulLoadGenerator.GenerateLoad"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.StatefulLoadGenerator.GenerateLoad">[docs]</a> <span class="k">class</span> <span class="nc">GenerateLoad</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="n">state_spec</span> <span class="o">=</span> <span class="n">userstate</span><span class="o">.</span><span class="n">CombiningValueStateSpec</span><span class="p">(</span> |
| <span class="s1">'bundles_remaining'</span><span class="p">,</span> <span class="n">combine_fn</span><span class="o">=</span><span class="nb">sum</span><span class="p">)</span> |
| <span class="n">timer_spec</span> <span class="o">=</span> <span class="n">userstate</span><span class="o">.</span><span class="n">TimerSpec</span><span class="p">(</span><span class="s1">'timer'</span><span class="p">,</span> <span class="n">userstate</span><span class="o">.</span><span class="n">TimeDomain</span><span class="o">.</span><span class="n">WATERMARK</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">num_records_per_key</span><span class="p">,</span> <span class="n">value_size</span><span class="p">,</span> <span class="n">bundle_size</span><span class="o">=</span><span class="mi">1000</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">num_records_per_key</span> <span class="o">=</span> <span class="n">num_records_per_key</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">payload</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">urandom</span><span class="p">(</span><span class="n">value_size</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">bundle_size</span> <span class="o">=</span> <span class="n">bundle_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">time_fn</span> <span class="o">=</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span> |
| |
| <div class="viewcode-block" id="StatefulLoadGenerator.GenerateLoad.process"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.StatefulLoadGenerator.GenerateLoad.process">[docs]</a> <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">records_remaining</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">StateParam</span><span class="p">(</span><span class="n">state_spec</span><span class="p">),</span> |
| <span class="n">timer</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimerParam</span><span class="p">(</span><span class="n">timer_spec</span><span class="p">)):</span> |
| <span class="n">records_remaining</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">num_records_per_key</span><span class="p">)</span> |
| <span class="n">timer</span><span class="o">.</span><span class="n">set</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="StatefulLoadGenerator.GenerateLoad.process_timer"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.StatefulLoadGenerator.GenerateLoad.process_timer">[docs]</a> <span class="nd">@userstate</span><span class="o">.</span><span class="n">on_timer</span><span class="p">(</span><span class="n">timer_spec</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">process_timer</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">key</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">KeyParam</span><span class="p">,</span> |
| <span class="n">records_remaining</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">StateParam</span><span class="p">(</span><span class="n">state_spec</span><span class="p">),</span> |
| <span class="n">timer</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimerParam</span><span class="p">(</span><span class="n">timer_spec</span><span class="p">)):</span> |
| <span class="n">cur_bundle_size</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">bundle_size</span><span class="p">,</span> <span class="n">records_remaining</span><span class="o">.</span><span class="n">read</span><span class="p">())</span> |
| <span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">cur_bundle_size</span><span class="p">):</span> |
| <span class="n">records_remaining</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="o">-</span><span class="mi">1</span><span class="p">)</span> |
| <span class="k">yield</span> <span class="n">key</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">payload</span> |
| <span class="k">if</span> <span class="n">records_remaining</span><span class="o">.</span><span class="n">read</span><span class="p">()</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">timer</span><span class="o">.</span><span class="n">set</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span></div></div> |
| |
| <div class="viewcode-block" id="StatefulLoadGenerator.expand"><a class="viewcode-back" href="../../../apache_beam.testing.synthetic_pipeline.html#apache_beam.testing.synthetic_pipeline.StatefulLoadGenerator.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">pbegin</span><span class="p">):</span> |
| <span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pbegin</span><span class="p">,</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PBegin</span><span class="p">),</span> <span class="p">(</span> |
| <span class="s1">'Input to transform must be a PBegin but found </span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="n">pbegin</span><span class="p">)</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">pbegin</span> |
| <span class="o">|</span> <span class="s1">'Impulse'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Impulse</span><span class="p">()</span> |
| <span class="o">|</span> <span class="s1">'GenerateKeys'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span> |
| <span class="n">StatefulLoadGenerator</span><span class="o">.</span><span class="n">GenerateKeys</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">num_keys</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">key_size</span><span class="p">))</span> |
| <span class="o">|</span> <span class="s1">'GenerateLoad'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span> |
| <span class="n">StatefulLoadGenerator</span><span class="o">.</span><span class="n">GenerateLoad</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">num_records</span> <span class="o">//</span> <span class="bp">self</span><span class="o">.</span><span class="n">num_keys</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">value_size</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> |