| |
| |
| <!DOCTYPE html> |
| <!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]--> |
| <!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]--> |
| <head> |
| <meta charset="utf-8"> |
| |
| <meta name="viewport" content="width=device-width, initial-scale=1.0"> |
| |
| <title>apache_beam.runners.interactive.caching.streaming_cache — 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.runners.interactive.caching.streaming_cache</li> |
| |
| |
| <li class="wy-breadcrumbs-aside"> |
| |
| </li> |
| |
| </ul> |
| |
| |
| <hr/> |
| </div> |
| <div role="main" class="document" itemscope="itemscope" itemtype="http://schema.org/Article"> |
| <div itemprop="articleBody"> |
| |
| <h1>Source code for apache_beam.runners.interactive.caching.streaming_cache</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="c1"># pytype: skip-file</span> |
| |
| <span class="kn">import</span> <span class="nn">logging</span> |
| <span class="kn">import</span> <span class="nn">os</span> |
| <span class="kn">import</span> <span class="nn">shutil</span> |
| <span class="kn">import</span> <span class="nn">tempfile</span> |
| <span class="kn">import</span> <span class="nn">time</span> |
| <span class="kn">import</span> <span class="nn">traceback</span> |
| <span class="kn">from</span> <span class="nn">collections</span> <span class="kn">import</span> <span class="n">OrderedDict</span> |
| <span class="c1"># We don't have an explicit pathlib dependency because this code only works with</span> |
| <span class="c1"># the interactive target installed which has an indirect dependency on pathlib</span> |
| <span class="c1"># through ipython>=5.9.0.</span> |
| <span class="kn">from</span> <span class="nn">pathlib</span> <span class="kn">import</span> <span class="n">Path</span> |
| |
| <span class="kn">from</span> <span class="nn">google.protobuf.message</span> <span class="kn">import</span> <span class="n">DecodeError</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">coders</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">beam_interactive_api_pb2</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">beam_runner_api_pb2</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.cache_manager</span> <span class="kn">import</span> <span class="n">CacheManager</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.cache_manager</span> <span class="kn">import</span> <span class="n">SafeFastPrimitivesCoder</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.runners.interactive.caching.cacheable</span> <span class="kn">import</span> <span class="n">CacheKey</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.testing.test_stream</span> <span class="kn">import</span> <span class="n">OutputFormat</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.testing.test_stream</span> <span class="kn">import</span> <span class="n">ReverseTestStream</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="kn">import</span> <span class="n">timestamp</span> |
| |
| <span class="n">_LOGGER</span> <span class="o">=</span> <span class="n">logging</span><span class="o">.</span><span class="n">getLogger</span><span class="p">(</span><span class="vm">__name__</span><span class="p">)</span> |
| |
| |
| <div class="viewcode-block" id="StreamingCacheSink"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCacheSink">[docs]</a><span class="k">class</span> <span class="nc">StreamingCacheSink</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 that writes TestStreamFile(Header|Records)s to file.</span> |
| |
| <span class="sd"> This transform takes in an arbitrary element stream and writes the list of</span> |
| <span class="sd"> TestStream events (as TestStreamFileRecords) to file. When replayed, this</span> |
| <span class="sd"> will produce the best-effort replay of the original job (e.g. some elements</span> |
| <span class="sd"> may be produced slightly out of order from the original stream).</span> |
| |
| <span class="sd"> Note that this PTransform is assumed to be only run on a single machine where</span> |
| <span class="sd"> the following assumptions are correct: elements come in ordered, no two</span> |
| <span class="sd"> transforms are writing to the same file. This PTransform is assumed to only</span> |
| <span class="sd"> run correctly with the DirectRunner.</span> |
| |
| <span class="sd"> TODO(https://github.com/apache/beam/issues/20002): Generalize this to more</span> |
| <span class="sd"> source/sink types aside from file based. Also, generalize to cases where</span> |
| <span class="sd"> there might be multiple workers writing to the same sink.</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">cache_dir</span><span class="p">,</span> |
| <span class="n">filename</span><span class="p">,</span> |
| <span class="n">sample_resolution_sec</span><span class="p">,</span> |
| <span class="n">coder</span><span class="o">=</span><span class="n">SafeFastPrimitivesCoder</span><span class="p">()):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span> <span class="o">=</span> <span class="n">cache_dir</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_filename</span> <span class="o">=</span> <span class="n">filename</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_sample_resolution_sec</span> <span class="o">=</span> <span class="n">sample_resolution_sec</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span> <span class="o">=</span> <span class="n">coder</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_path</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_filename</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">path</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns the path the sink leads to."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_path</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">size_in_bytes</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns the space usage in bytes of the sink."""</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">os</span><span class="o">.</span><span class="n">stat</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_path</span><span class="p">)</span><span class="o">.</span><span class="n">st_size</span> |
| <span class="k">except</span> <span class="ne">OSError</span><span class="p">:</span> |
| <span class="n">_LOGGER</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span> |
| <span class="s1">'Failed to calculate cache size for file </span><span class="si">%s</span><span class="s1">, the file might have not '</span> |
| <span class="s1">'been created yet. Return 0. </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">_path</span><span class="p">,</span> |
| <span class="n">traceback</span><span class="o">.</span><span class="n">format_exc</span><span class="p">())</span> |
| <span class="k">return</span> <span class="mi">0</span> |
| |
| <div class="viewcode-block" id="StreamingCacheSink.expand"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCacheSink.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span> |
| <span class="k">class</span> <span class="nc">StreamingWriteToText</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">"""DoFn that performs the writing.</span> |
| |
| <span class="sd"> Note that the other file writing methods cannot be used in streaming</span> |
| <span class="sd"> contexts.</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">full_path</span><span class="p">,</span> <span class="n">coder</span><span class="o">=</span><span class="n">SafeFastPrimitivesCoder</span><span class="p">()):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_full_path</span> <span class="o">=</span> <span class="n">full_path</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span> <span class="o">=</span> <span class="n">coder</span> |
| |
| <span class="c1"># Try and make the given path.</span> |
| <span class="n">Path</span><span class="p">(</span><span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">dirname</span><span class="p">(</span><span class="n">full_path</span><span class="p">))</span><span class="o">.</span><span class="n">mkdir</span><span class="p">(</span><span class="n">parents</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">exist_ok</span><span class="o">=</span><span class="kc">True</span><span class="p">)</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="c1"># Open the file for 'append-mode' and writing 'bytes'.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_fh</span> <span class="o">=</span> <span class="nb">open</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_full_path</span><span class="p">,</span> <span class="s1">'ab'</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="bp">self</span><span class="o">.</span><span class="n">_fh</span><span class="o">.</span><span class="n">close</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">e</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Appends the given element to the file.</span> |
| <span class="sd"> """</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_fh</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="n">e</span><span class="p">)</span> <span class="o">+</span> <span class="sa">b</span><span class="s1">'</span><span class="se">\n</span><span class="s1">'</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">pcoll</span> |
| <span class="o">|</span> <span class="n">ReverseTestStream</span><span class="p">(</span> |
| <span class="n">output_tag</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_filename</span><span class="p">,</span> |
| <span class="n">sample_resolution_sec</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_sample_resolution_sec</span><span class="p">,</span> |
| <span class="n">output_format</span><span class="o">=</span><span class="n">OutputFormat</span><span class="o">.</span><span class="n">SERIALIZED_TEST_STREAM_FILE_RECORDS</span><span class="p">,</span> |
| <span class="n">coder</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</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">StreamingWriteToText</span><span class="p">(</span><span class="n">full_path</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_path</span><span class="p">,</span> <span class="n">coder</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="p">)))</span></div></div> |
| |
| |
| <div class="viewcode-block" id="StreamingCacheSource"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCacheSource">[docs]</a><span class="k">class</span> <span class="nc">StreamingCacheSource</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""A class that reads and parses TestStreamFile(Header|Reader)s.</span> |
| |
| <span class="sd"> This source operates in the following way:</span> |
| |
| <span class="sd"> 1. Wait for up to `timeout_secs` for the file to be available.</span> |
| <span class="sd"> 2. Read, parse, and emit the entire contents of the file</span> |
| <span class="sd"> 3. Wait for more events to come or until `is_cache_complete` returns True</span> |
| <span class="sd"> 4. If there are more events, then go to 2</span> |
| <span class="sd"> 5. Otherwise, stop emitting.</span> |
| |
| <span class="sd"> This class is used to read from file and send its to the TestStream via the</span> |
| <span class="sd"> StreamingCacheManager.Reader.</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">cache_dir</span><span class="p">,</span> <span class="n">labels</span><span class="p">,</span> <span class="n">is_cache_complete</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">coder</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">coder</span><span class="p">:</span> |
| <span class="n">coder</span> <span class="o">=</span> <span class="n">SafeFastPrimitivesCoder</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">is_cache_complete</span><span class="p">:</span> |
| <span class="n">is_cache_complete</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">_</span><span class="p">:</span> <span class="kc">True</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span> <span class="o">=</span> <span class="n">cache_dir</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span> <span class="o">=</span> <span class="n">coder</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_labels</span> <span class="o">=</span> <span class="n">labels</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_path</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">_labels</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_is_cache_complete</span> <span class="o">=</span> <span class="n">is_cache_complete</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_id</span> <span class="o">=</span> <span class="n">CacheKey</span><span class="o">.</span><span class="n">from_str</span><span class="p">(</span><span class="n">labels</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">pipeline_id</span> |
| |
| <span class="k">def</span> <span class="nf">_wait_until_file_exists</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">timeout_secs</span><span class="o">=</span><span class="mi">30</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Blocks until the file exists for a maximum of timeout_secs.</span> |
| <span class="sd"> """</span> |
| <span class="c1"># Wait for up to `timeout_secs` for the file to be available.</span> |
| <span class="n">start</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">while</span> <span class="ow">not</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_path</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="mi">1</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">time</span><span class="o">.</span><span class="n">time</span><span class="p">()</span> <span class="o">-</span> <span class="n">start</span> <span class="o">></span> <span class="n">timeout_secs</span><span class="p">:</span> |
| <span class="n">pcollection_var</span> <span class="o">=</span> <span class="n">CacheKey</span><span class="o">.</span><span class="n">from_str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_labels</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">var</span> |
| <span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span> |
| <span class="s1">'Timed out waiting for cache file for PCollection `</span><span class="si">{}</span><span class="s1">` to be '</span> |
| <span class="s1">'available with path </span><span class="si">{}</span><span class="s1">.'</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">pcollection_var</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_path</span><span class="p">))</span> |
| <span class="k">return</span> <span class="nb">open</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_path</span><span class="p">,</span> <span class="n">mode</span><span class="o">=</span><span class="s1">'rb'</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_emit_from_file</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">fh</span><span class="p">,</span> <span class="n">tail</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Emits the TestStreamFile(Header|Record)s from file.</span> |
| |
| <span class="sd"> This returns a generator to be able to read all lines from the given file.</span> |
| <span class="sd"> If `tail` is True, then it will wait until the cache is complete to exit.</span> |
| <span class="sd"> Otherwise, it will read the file only once.</span> |
| <span class="sd"> """</span> |
| <span class="c1"># Always read at least once to read the whole file.</span> |
| <span class="k">while</span> <span class="kc">True</span><span class="p">:</span> |
| <span class="n">pos</span> <span class="o">=</span> <span class="n">fh</span><span class="o">.</span><span class="n">tell</span><span class="p">()</span> |
| <span class="n">line</span> <span class="o">=</span> <span class="n">fh</span><span class="o">.</span><span class="n">readline</span><span class="p">()</span> |
| |
| <span class="c1"># Check if we are at EOF or if we have an incomplete line.</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">line</span> <span class="ow">or</span> <span class="p">(</span><span class="n">line</span> <span class="ow">and</span> <span class="n">line</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> <span class="o">!=</span> <span class="sa">b</span><span class="s1">'</span><span class="se">\n</span><span class="s1">'</span><span class="p">[</span><span class="mi">0</span><span class="p">]):</span> |
| <span class="c1"># Read at least the first line to get the header.</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">tail</span> <span class="ow">and</span> <span class="n">pos</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">break</span> |
| |
| <span class="c1"># Complete reading only when the cache is complete.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_cache_complete</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pipeline_id</span><span class="p">):</span> |
| <span class="k">break</span> |
| |
| <span class="c1"># Otherwise wait for new data in the file to be written.</span> |
| <span class="n">time</span><span class="o">.</span><span class="n">sleep</span><span class="p">(</span><span class="mf">0.5</span><span class="p">)</span> |
| <span class="n">fh</span><span class="o">.</span><span class="n">seek</span><span class="p">(</span><span class="n">pos</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># The first line at pos = 0 is always the header. Read the line without</span> |
| <span class="c1"># the new line.</span> |
| <span class="n">to_decode</span> <span class="o">=</span> <span class="n">line</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="n">pos</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">proto_cls</span> <span class="o">=</span> <span class="n">beam_interactive_api_pb2</span><span class="o">.</span><span class="n">TestStreamFileHeader</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">proto_cls</span> <span class="o">=</span> <span class="n">beam_interactive_api_pb2</span><span class="o">.</span><span class="n">TestStreamFileRecord</span> |
| <span class="n">msg</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_try_parse_as</span><span class="p">(</span><span class="n">proto_cls</span><span class="p">,</span> <span class="n">to_decode</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">msg</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">msg</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">break</span> |
| |
| <span class="k">def</span> <span class="nf">_try_parse_as</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">proto_cls</span><span class="p">,</span> <span class="n">to_decode</span><span class="p">):</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">msg</span> <span class="o">=</span> <span class="n">proto_cls</span><span class="p">()</span> |
| <span class="n">msg</span><span class="o">.</span><span class="n">ParseFromString</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">decode</span><span class="p">(</span><span class="n">to_decode</span><span class="p">))</span> |
| <span class="k">except</span> <span class="n">DecodeError</span><span class="p">:</span> |
| <span class="n">_LOGGER</span><span class="o">.</span><span class="n">error</span><span class="p">(</span> |
| <span class="s1">'Could not parse as </span><span class="si">%s</span><span class="s1">. This can indicate that the cache is '</span> |
| <span class="s1">'corruputed. Please restart the kernel. '</span> |
| <span class="s1">'</span><span class="se">\n</span><span class="s1">file: </span><span class="si">%s</span><span class="s1"> </span><span class="se">\n</span><span class="s1">message: </span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> |
| <span class="n">proto_cls</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_path</span><span class="p">,</span> |
| <span class="n">to_decode</span><span class="p">)</span> |
| <span class="n">msg</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">return</span> <span class="n">msg</span> |
| |
| <div class="viewcode-block" id="StreamingCacheSource.read"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCacheSource.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">tail</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Reads all TestStreamFile(Header|TestStreamFileRecord)s from file.</span> |
| |
| <span class="sd"> This returns a generator to be able to read all lines from the given file.</span> |
| <span class="sd"> If `tail` is True, then it will wait until the cache is complete to exit.</span> |
| <span class="sd"> Otherwise, it will read the file only once.</span> |
| <span class="sd"> """</span> |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_wait_until_file_exists</span><span class="p">()</span> <span class="k">as</span> <span class="n">f</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">e</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_emit_from_file</span><span class="p">(</span><span class="n">f</span><span class="p">,</span> <span class="n">tail</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">e</span></div></div> |
| |
| |
| <span class="c1"># TODO(victorhc): Add support for cache_dir locations that are on GCS</span> |
| <div class="viewcode-block" id="StreamingCache"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache">[docs]</a><span class="k">class</span> <span class="nc">StreamingCache</span><span class="p">(</span><span class="n">CacheManager</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Abstraction that holds the logic for reading and writing to cache.</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">cache_dir</span><span class="p">,</span> |
| <span class="n">is_cache_complete</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">sample_resolution_sec</span><span class="o">=</span><span class="mf">0.1</span><span class="p">,</span> |
| <span class="n">saved_pcoders</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_sample_resolution_sec</span> <span class="o">=</span> <span class="n">sample_resolution_sec</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_is_cache_complete</span> <span class="o">=</span> <span class="n">is_cache_complete</span> |
| |
| <span class="k">if</span> <span class="n">cache_dir</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span> <span class="o">=</span> <span class="n">cache_dir</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span> <span class="o">=</span> <span class="n">tempfile</span><span class="o">.</span><span class="n">mkdtemp</span><span class="p">(</span> |
| <span class="n">prefix</span><span class="o">=</span><span class="s1">'ib-'</span><span class="p">,</span> <span class="nb">dir</span><span class="o">=</span><span class="n">os</span><span class="o">.</span><span class="n">environ</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'TEST_TMPDIR'</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span> |
| |
| <span class="c1"># List of saved pcoders keyed by PCollection path. It is OK to keep this</span> |
| <span class="c1"># list in memory because once FileBasedCacheManager object is</span> |
| <span class="c1"># destroyed/re-created it loses the access to previously written cache</span> |
| <span class="c1"># objects anyways even if cache_dir already exists. In other words,</span> |
| <span class="c1"># it is not possible to resume execution of Beam pipeline from the</span> |
| <span class="c1"># saved cache if FileBasedCacheManager has been reset.</span> |
| <span class="c1">#</span> |
| <span class="c1"># However, if we are to implement better cache persistence, one needs</span> |
| <span class="c1"># to take care of keeping consistency between the cached PCollection</span> |
| <span class="c1"># and its PCoder type.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_saved_pcoders</span> <span class="o">=</span> <span class="n">saved_pcoders</span> <span class="ow">or</span> <span class="p">{}</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_default_pcoder</span> <span class="o">=</span> <span class="n">SafeFastPrimitivesCoder</span><span class="p">()</span> |
| |
| <span class="c1"># The sinks to capture data from capturable sources.</span> |
| <span class="c1"># Dict([str, StreamingCacheSink])</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_capture_sinks</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_capture_keys</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span> |
| |
| <div class="viewcode-block" id="StreamingCache.size"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.size">[docs]</a> <span class="k">def</span> <span class="nf">size</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">):</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="o">*</span><span class="n">labels</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">getsize</span><span class="p">(</span><span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">))</span> |
| <span class="k">return</span> <span class="mi">0</span></div> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">capture_size</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">sum</span><span class="p">([</span><span class="n">sink</span><span class="o">.</span><span class="n">size_in_bytes</span> <span class="k">for</span> <span class="n">_</span><span class="p">,</span> <span class="n">sink</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_capture_sinks</span><span class="o">.</span><span class="n">items</span><span class="p">()])</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">capture_paths</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_capture_sinks</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">capture_keys</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">_capture_keys</span> |
| |
| <div class="viewcode-block" id="StreamingCache.exists"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.exists">[docs]</a> <span class="k">def</span> <span class="nf">exists</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">labels</span> <span class="ow">and</span> <span class="nb">any</span><span class="p">(</span><span class="n">labels</span><span class="p">):</span> |
| <span class="n">path</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="n">path</span><span class="p">)</span> |
| <span class="k">return</span> <span class="kc">False</span></div> |
| |
| <span class="c1"># TODO(srohde): Modify this to return the correct version.</span> |
| <div class="viewcode-block" id="StreamingCache.read"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.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="o">*</span><span class="n">labels</span><span class="p">,</span> <span class="o">**</span><span class="n">args</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns a generator to read all records from file."""</span> |
| <span class="n">tail</span> <span class="o">=</span> <span class="n">args</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="s1">'tail'</span><span class="p">,</span> <span class="kc">False</span><span class="p">)</span> |
| |
| <span class="c1"># Only immediately return when the file doesn't exist when the user wants a</span> |
| <span class="c1"># snapshot of the cache (when tail is false).</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="o">*</span><span class="n">labels</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">tail</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">iter</span><span class="p">([]),</span> <span class="o">-</span><span class="mi">1</span> |
| |
| <span class="n">reader</span> <span class="o">=</span> <span class="n">StreamingCacheSource</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> |
| <span class="n">labels</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_is_cache_complete</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">load_pcoder</span><span class="p">(</span><span class="o">*</span><span class="n">labels</span><span class="p">))</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="n">tail</span><span class="o">=</span><span class="n">tail</span><span class="p">)</span> |
| |
| <span class="c1"># Return an empty iterator if there is nothing in the file yet. This can</span> |
| <span class="c1"># only happen when tail is False.</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">header</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">reader</span><span class="p">)</span> |
| <span class="k">except</span> <span class="ne">StopIteration</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">iter</span><span class="p">([]),</span> <span class="o">-</span><span class="mi">1</span> |
| <span class="k">return</span> <span class="n">StreamingCache</span><span class="o">.</span><span class="n">Reader</span><span class="p">([</span><span class="n">header</span><span class="p">],</span> <span class="p">[</span><span class="n">reader</span><span class="p">])</span><span class="o">.</span><span class="n">read</span><span class="p">(),</span> <span class="mi">1</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.read_multiple"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.read_multiple">[docs]</a> <span class="k">def</span> <span class="nf">read_multiple</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">labels</span><span class="p">,</span> <span class="n">tail</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns a generator to read all records from file.</span> |
| |
| <span class="sd"> Does tail until the cache is complete. This is because it is used in the</span> |
| <span class="sd"> TestStreamServiceController to read from file which is only used during</span> |
| <span class="sd"> pipeline runtime which needs to block.</span> |
| <span class="sd"> """</span> |
| <span class="n">readers</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">StreamingCacheSource</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="n">l</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_cache_complete</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">load_pcoder</span><span class="p">(</span><span class="o">*</span><span class="n">l</span><span class="p">))</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="n">tail</span><span class="o">=</span><span class="n">tail</span><span class="p">)</span> <span class="k">for</span> <span class="n">l</span> <span class="ow">in</span> <span class="n">labels</span> |
| <span class="p">]</span> |
| <span class="n">headers</span> <span class="o">=</span> <span class="p">[</span><span class="nb">next</span><span class="p">(</span><span class="n">r</span><span class="p">)</span> <span class="k">for</span> <span class="n">r</span> <span class="ow">in</span> <span class="n">readers</span><span class="p">]</span> |
| <span class="k">return</span> <span class="n">StreamingCache</span><span class="o">.</span><span class="n">Reader</span><span class="p">(</span><span class="n">headers</span><span class="p">,</span> <span class="n">readers</span><span class="p">)</span><span class="o">.</span><span class="n">read</span><span class="p">()</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.write"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.write">[docs]</a> <span class="k">def</span> <span class="nf">write</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">values</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Writes the given values to cache.</span> |
| <span class="sd"> """</span> |
| <span class="n">directory</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">])</span> |
| <span class="n">filepath</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">directory</span><span class="p">,</span> <span class="n">labels</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="ow">not</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="n">directory</span><span class="p">):</span> |
| <span class="n">os</span><span class="o">.</span><span class="n">makedirs</span><span class="p">(</span><span class="n">directory</span><span class="p">)</span> |
| <span class="k">with</span> <span class="nb">open</span><span class="p">(</span><span class="n">filepath</span><span class="p">,</span> <span class="s1">'ab'</span><span class="p">)</span> <span class="k">as</span> <span class="n">f</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">values</span><span class="p">:</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> |
| <span class="p">(</span><span class="n">beam_interactive_api_pb2</span><span class="o">.</span><span class="n">TestStreamFileHeader</span><span class="p">,</span> |
| <span class="n">beam_interactive_api_pb2</span><span class="o">.</span><span class="n">TestStreamFileRecord</span><span class="p">)):</span> |
| <span class="n">val</span> <span class="o">=</span> <span class="n">v</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span> |
| <span class="s1">'Values given to streaming cache should be either '</span> |
| <span class="s1">'TestStreamFileHeader or TestStreamFileRecord.'</span><span class="p">)</span> |
| <span class="n">f</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">load_pcoder</span><span class="p">(</span><span class="o">*</span><span class="n">labels</span><span class="p">)</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="n">val</span><span class="p">)</span> <span class="o">+</span> <span class="sa">b</span><span class="s1">'</span><span class="se">\n</span><span class="s1">'</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.clear"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.clear">[docs]</a> <span class="k">def</span> <span class="nf">clear</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">):</span> |
| <span class="n">directory</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">])</span> |
| <span class="n">filepath</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">directory</span><span class="p">,</span> <span class="n">labels</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">])</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_capture_keys</span><span class="o">.</span><span class="n">discard</span><span class="p">(</span><span class="n">labels</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="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="n">filepath</span><span class="p">):</span> |
| <span class="n">os</span><span class="o">.</span><span class="n">remove</span><span class="p">(</span><span class="n">filepath</span><span class="p">)</span> |
| <span class="k">return</span> <span class="kc">True</span> |
| <span class="k">return</span> <span class="kc">False</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.source"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.source">[docs]</a> <span class="k">def</span> <span class="nf">source</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns the StreamingCacheManager source.</span> |
| |
| <span class="sd"> This is beam.Impulse() because unbounded sources will be marked with this</span> |
| <span class="sd"> and then the PipelineInstrument will replace these with a TestStream.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">Impulse</span><span class="p">()</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.sink"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.sink">[docs]</a> <span class="k">def</span> <span class="nf">sink</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">labels</span><span class="p">,</span> <span class="n">is_capture</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Returns a StreamingCacheSink to write elements to file.</span> |
| |
| <span class="sd"> Note that this is assumed to only work in the DirectRunner as the underlying</span> |
| <span class="sd"> StreamingCacheSink assumes a single machine to have correct element</span> |
| <span class="sd"> ordering.</span> |
| <span class="sd"> """</span> |
| <span class="n">filename</span> <span class="o">=</span> <span class="n">labels</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> |
| <span class="n">cache_dir</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">])</span> |
| <span class="n">sink</span> <span class="o">=</span> <span class="n">StreamingCacheSink</span><span class="p">(</span> |
| <span class="n">cache_dir</span><span class="p">,</span> |
| <span class="n">filename</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_sample_resolution_sec</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">load_pcoder</span><span class="p">(</span><span class="o">*</span><span class="n">labels</span><span class="p">))</span> |
| <span class="k">if</span> <span class="n">is_capture</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_capture_sinks</span><span class="p">[</span><span class="n">sink</span><span class="o">.</span><span class="n">path</span><span class="p">]</span> <span class="o">=</span> <span class="n">sink</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_capture_keys</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">filename</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">sink</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.save_pcoder"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.save_pcoder">[docs]</a> <span class="k">def</span> <span class="nf">save_pcoder</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoder</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_saved_pcoders</span><span class="p">[</span><span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">)]</span> <span class="o">=</span> <span class="n">pcoder</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.load_pcoder"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.load_pcoder">[docs]</a> <span class="k">def</span> <span class="nf">load_pcoder</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">):</span> |
| <span class="n">saved_pcoder</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_saved_pcoders</span><span class="o">.</span><span class="n">get</span><span class="p">(</span> |
| <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="o">*</span><span class="n">labels</span><span class="p">),</span> <span class="kc">None</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">saved_pcoder</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">saved_pcoder</span><span class="p">,</span> |
| <span class="n">coders</span><span class="o">.</span><span class="n">FastPrimitivesCoder</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_default_pcoder</span> |
| <span class="k">return</span> <span class="n">saved_pcoder</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.cleanup"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.cleanup">[docs]</a> <span class="k">def</span> <span class="nf">cleanup</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| |
| <span class="k">if</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">exists</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">):</span> |
| |
| <span class="k">def</span> <span class="nf">on_fail_to_cleanup</span><span class="p">(</span><span class="n">function</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">excinfo</span><span class="p">):</span> |
| <span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span> |
| <span class="s1">'Failed to clean up temporary files: </span><span class="si">%s</span><span class="s1">. You may'</span> |
| <span class="s1">'manually delete them if necessary. Error was: </span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> |
| <span class="n">path</span><span class="p">,</span> |
| <span class="n">excinfo</span><span class="p">)</span> |
| |
| <span class="n">shutil</span><span class="o">.</span><span class="n">rmtree</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_cache_dir</span><span class="p">,</span> <span class="n">onerror</span><span class="o">=</span><span class="n">on_fail_to_cleanup</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_saved_pcoders</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_capture_sinks</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_capture_keys</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span></div> |
| |
| <div class="viewcode-block" id="StreamingCache.Reader"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.Reader">[docs]</a> <span class="k">class</span> <span class="nc">Reader</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Abstraction that reads from PCollection readers.</span> |
| |
| <span class="sd"> This class is an Abstraction layer over multiple PCollection readers to be</span> |
| <span class="sd"> used for supplying a TestStream service with events.</span> |
| |
| <span class="sd"> This class is also responsible for holding the state of the clock, injecting</span> |
| <span class="sd"> clock advancement events, and watermark advancement events.</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">headers</span><span class="p">,</span> <span class="n">readers</span><span class="p">):</span> |
| <span class="c1"># This timestamp is used as the monotonic clock to order events in the</span> |
| <span class="c1"># replay.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_monotonic_clock</span> <span class="o">=</span> <span class="n">timestamp</span><span class="o">.</span><span class="n">Timestamp</span><span class="o">.</span><span class="n">of</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span> |
| |
| <span class="c1"># The PCollection cache readers.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_readers</span> <span class="o">=</span> <span class="p">{}</span> |
| |
| <span class="c1"># The file headers that are metadata for that particular PCollection.</span> |
| <span class="c1"># The header allows for metadata about an entire stream, so that the data</span> |
| <span class="c1"># isn't copied per record.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_headers</span> <span class="o">=</span> <span class="p">{</span><span class="n">header</span><span class="o">.</span><span class="n">tag</span><span class="p">:</span> <span class="n">header</span> <span class="k">for</span> <span class="n">header</span> <span class="ow">in</span> <span class="n">headers</span><span class="p">}</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_readers</span> <span class="o">=</span> <span class="n">OrderedDict</span><span class="p">(</span> |
| <span class="p">((</span><span class="n">h</span><span class="o">.</span><span class="n">tag</span><span class="p">,</span> <span class="n">r</span><span class="p">)</span> <span class="k">for</span> <span class="p">(</span><span class="n">h</span><span class="p">,</span> <span class="n">r</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">headers</span><span class="p">,</span> <span class="n">readers</span><span class="p">)))</span> |
| |
| <span class="c1"># The most recently read timestamp per tag.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_stream_times</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">tag</span><span class="p">:</span> <span class="n">timestamp</span><span class="o">.</span><span class="n">Timestamp</span><span class="p">(</span><span class="n">seconds</span><span class="o">=</span><span class="mi">0</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_headers</span> |
| <span class="p">}</span> |
| |
| <span class="k">def</span> <span class="nf">_test_stream_events_before_target</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">target_timestamp</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Reads the next iteration of elements from each stream.</span> |
| |
| <span class="sd"> Retrieves an element from each stream iff the most recently read timestamp</span> |
| <span class="sd"> from that stream is less than the target_timestamp. Since the amount of</span> |
| <span class="sd"> events may not fit into memory, this StreamingCache reads at most one</span> |
| <span class="sd"> element from each stream at a time.</span> |
| <span class="sd"> """</span> |
| <span class="n">records</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">r</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_readers</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="c1"># The target_timestamp is the maximum timestamp that was read from the</span> |
| <span class="c1"># stream. Some readers may have elements that are less than this. Thus,</span> |
| <span class="c1"># we skip all readers that already have elements that are at this</span> |
| <span class="c1"># timestamp so that we don't read everything into memory.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_stream_times</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span> <span class="o">>=</span> <span class="n">target_timestamp</span><span class="p">:</span> |
| <span class="k">continue</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">record</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">r</span><span class="p">)</span><span class="o">.</span><span class="n">recorded_event</span> |
| <span class="k">if</span> <span class="n">record</span><span class="o">.</span><span class="n">HasField</span><span class="p">(</span><span class="s1">'processing_time_event'</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_stream_times</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span> <span class="o">+=</span> <span class="n">timestamp</span><span class="o">.</span><span class="n">Duration</span><span class="p">(</span> |
| <span class="n">micros</span><span class="o">=</span><span class="n">record</span><span class="o">.</span><span class="n">processing_time_event</span><span class="o">.</span><span class="n">advance_duration</span><span class="p">)</span> |
| <span class="n">records</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">tag</span><span class="p">,</span> <span class="n">record</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_stream_times</span><span class="p">[</span><span class="n">tag</span><span class="p">]))</span> |
| <span class="k">except</span> <span class="ne">StopIteration</span><span class="p">:</span> |
| <span class="k">pass</span> |
| <span class="k">return</span> <span class="n">records</span> |
| |
| <span class="k">def</span> <span class="nf">_merge_sort</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">previous_events</span><span class="p">,</span> <span class="n">new_events</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">sorted</span><span class="p">(</span> |
| <span class="n">previous_events</span> <span class="o">+</span> <span class="n">new_events</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">[</span><span class="mi">2</span><span class="p">],</span> <span class="n">reverse</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_min_timestamp_of</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">events</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">events</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">][</span><span class="mi">2</span><span class="p">]</span> <span class="k">if</span> <span class="n">events</span> <span class="k">else</span> <span class="n">timestamp</span><span class="o">.</span><span class="n">MAX_TIMESTAMP</span> |
| |
| <span class="k">def</span> <span class="nf">_event_stream_caught_up_to_target</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">events</span><span class="p">,</span> <span class="n">target_timestamp</span><span class="p">):</span> |
| <span class="n">empty_events</span> <span class="o">=</span> <span class="ow">not</span> <span class="n">events</span> |
| <span class="n">stream_is_past_target</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_timestamp_of</span><span class="p">(</span><span class="n">events</span><span class="p">)</span> <span class="o">></span> <span class="n">target_timestamp</span> |
| <span class="k">return</span> <span class="n">empty_events</span> <span class="ow">or</span> <span class="n">stream_is_past_target</span> |
| |
| <div class="viewcode-block" id="StreamingCache.Reader.read"><a class="viewcode-back" href="../../../../../apache_beam.runners.interactive.caching.streaming_cache.html#apache_beam.runners.interactive.caching.streaming_cache.StreamingCache.Reader.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="w"> </span><span class="sd">"""Reads records from PCollection readers.</span> |
| <span class="sd"> """</span> |
| |
| <span class="c1"># The largest timestamp read from the different streams.</span> |
| <span class="n">target_timestamp</span> <span class="o">=</span> <span class="n">timestamp</span><span class="o">.</span><span class="n">MAX_TIMESTAMP</span> |
| |
| <span class="c1"># The events from last iteration that are past the target timestamp.</span> |
| <span class="n">unsent_events</span> <span class="o">=</span> <span class="p">[]</span> |
| |
| <span class="c1"># Emit events until all events have been read.</span> |
| <span class="k">while</span> <span class="kc">True</span><span class="p">:</span> |
| <span class="c1"># Read the next set of events. The read events will most likely be</span> |
| <span class="c1"># out of order if there are multiple readers. Here we sort them into</span> |
| <span class="c1"># a more manageable state.</span> |
| <span class="n">new_events</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_test_stream_events_before_target</span><span class="p">(</span><span class="n">target_timestamp</span><span class="p">)</span> |
| <span class="n">events_to_send</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_merge_sort</span><span class="p">(</span><span class="n">unsent_events</span><span class="p">,</span> <span class="n">new_events</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">events_to_send</span><span class="p">:</span> |
| <span class="k">break</span> |
| |
| <span class="c1"># Get the next largest timestamp in the stream. This is used as the</span> |
| <span class="c1"># timestamp for readers to "catch-up" to. This will only read from</span> |
| <span class="c1"># readers with a timestamp less than this.</span> |
| <span class="n">target_timestamp</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_timestamp_of</span><span class="p">(</span><span class="n">events_to_send</span><span class="p">)</span> |
| |
| <span class="c1"># Loop through the elements with the correct timestamp.</span> |
| <span class="k">while</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_event_stream_caught_up_to_target</span><span class="p">(</span><span class="n">events_to_send</span><span class="p">,</span> |
| <span class="n">target_timestamp</span><span class="p">):</span> |
| |
| <span class="c1"># First advance the clock to match the time of the stream. This has</span> |
| <span class="c1"># a side-effect of also advancing this cache's clock.</span> |
| <span class="n">tag</span><span class="p">,</span> <span class="n">r</span><span class="p">,</span> <span class="n">curr_timestamp</span> <span class="o">=</span> <span class="n">events_to_send</span><span class="o">.</span><span class="n">pop</span><span class="p">()</span> |
| <span class="k">if</span> <span class="n">curr_timestamp</span> <span class="o">></span> <span class="bp">self</span><span class="o">.</span><span class="n">_monotonic_clock</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="bp">self</span><span class="o">.</span><span class="n">_advance_processing_time</span><span class="p">(</span><span class="n">curr_timestamp</span><span class="p">)</span> |
| |
| <span class="c1"># Then, send either a new element or watermark.</span> |
| <span class="k">if</span> <span class="n">r</span><span class="o">.</span><span class="n">HasField</span><span class="p">(</span><span class="s1">'element_event'</span><span class="p">):</span> |
| <span class="n">r</span><span class="o">.</span><span class="n">element_event</span><span class="o">.</span><span class="n">tag</span> <span class="o">=</span> <span class="n">tag</span> |
| <span class="k">yield</span> <span class="n">r</span> |
| <span class="k">elif</span> <span class="n">r</span><span class="o">.</span><span class="n">HasField</span><span class="p">(</span><span class="s1">'watermark_event'</span><span class="p">):</span> |
| <span class="n">r</span><span class="o">.</span><span class="n">watermark_event</span><span class="o">.</span><span class="n">tag</span> <span class="o">=</span> <span class="n">tag</span> |
| <span class="k">yield</span> <span class="n">r</span> |
| <span class="n">unsent_events</span> <span class="o">=</span> <span class="n">events_to_send</span> |
| <span class="n">target_timestamp</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_timestamp_of</span><span class="p">(</span><span class="n">unsent_events</span><span class="p">)</span></div> |
| |
| <span class="k">def</span> <span class="nf">_advance_processing_time</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">new_timestamp</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Advances the internal clock and returns an AdvanceProcessingTime event.</span> |
| <span class="sd"> """</span> |
| <span class="n">advancy_by</span> <span class="o">=</span> <span class="n">new_timestamp</span><span class="o">.</span><span class="n">micros</span> <span class="o">-</span> <span class="bp">self</span><span class="o">.</span><span class="n">_monotonic_clock</span><span class="o">.</span><span class="n">micros</span> |
| <span class="n">e</span> <span class="o">=</span> <span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">TestStreamPayload</span><span class="o">.</span><span class="n">Event</span><span class="p">(</span> |
| <span class="n">processing_time_event</span><span class="o">=</span><span class="n">beam_runner_api_pb2</span><span class="o">.</span><span class="n">TestStreamPayload</span><span class="o">.</span><span class="n">Event</span><span class="o">.</span> |
| <span class="n">AdvanceProcessingTime</span><span class="p">(</span><span class="n">advance_duration</span><span class="o">=</span><span class="n">advancy_by</span><span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_monotonic_clock</span> <span class="o">=</span> <span class="n">new_timestamp</span> |
| <span class="k">return</span> <span class="n">e</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> |