blob: 7328bdcf22060b0a182bbc30639b09e178309404 [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.coders.slow_stream &mdash; Apache Beam documentation</title>
<link rel="stylesheet" href="../../../_static/css/theme.css" type="text/css" />
<link rel="index" title="Index"
href="../../../genindex.html"/>
<link rel="search" title="Search" href="../../../search.html"/>
<link rel="top" title="Apache Beam documentation" href="../../../index.html"/>
<link rel="up" title="Module code" href="../../index.html"/>
<script src="../../../_static/js/modernizr.min.js"></script>
</head>
<body class="wy-body-for-nav" role="document">
<div class="wy-grid-for-nav">
<nav data-toggle="wy-nav-shift" class="wy-nav-side">
<div class="wy-side-scroll">
<div class="wy-side-nav-search">
<a href="../../../index.html" class="icon icon-home"> Apache Beam
</a>
<div role="search">
<form id="rtd-search-form" class="wy-form" action="../../../search.html" method="get">
<input type="text" name="q" placeholder="Search docs" />
<input type="hidden" name="check_keywords" value="yes" />
<input type="hidden" name="area" value="default" />
</form>
</div>
</div>
<div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation">
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.coders.html">apache_beam.coders package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.internal.html">apache_beam.internal package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.io.html">apache_beam.io package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.metrics.html">apache_beam.metrics package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.options.html">apache_beam.options package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.portability.html">apache_beam.portability package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.runners.html">apache_beam.runners package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.testing.html">apache_beam.testing package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.tools.html">apache_beam.tools package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.transforms.html">apache_beam.transforms package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.typehints.html">apache_beam.typehints package</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.utils.html">apache_beam.utils package</a></li>
</ul>
<ul>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.error.html">apache_beam.error module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li>
<li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.version.html">apache_beam.version module</a></li>
</ul>
</div>
</div>
</nav>
<section data-toggle="wy-nav-shift" class="wy-nav-content-wrap">
<nav class="wy-nav-top" role="navigation" aria-label="top navigation">
<i data-toggle="wy-nav-top" class="fa fa-bars"></i>
<a href="../../../index.html">Apache Beam</a>
</nav>
<div class="wy-nav-content">
<div class="rst-content">
<div role="navigation" aria-label="breadcrumbs navigation">
<ul class="wy-breadcrumbs">
<li><a href="../../../index.html">Docs</a> &raquo;</li>
<li><a href="../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.coders.slow_stream</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.coders.slow_stream</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="sd">&quot;&quot;&quot;A pure Python implementation of stream.pyx.</span>
<span class="sd">For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="kn">import</span> <span class="n">absolute_import</span>
<span class="kn">import</span> <span class="nn">struct</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="kn">import</span> <span class="nb">chr</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="kn">import</span> <span class="nb">object</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">List</span>
<div class="viewcode-block" id="OutputStream"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream">[docs]</a><span class="k">class</span> <span class="nc">OutputStream</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> A pure Python implementation of stream.OutputStream.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data</span> <span class="o">=</span> <span class="p">[]</span> <span class="c1"># type: List[bytes]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">byte_count</span> <span class="o">=</span> <span class="mi">0</span>
<div class="viewcode-block" id="OutputStream.write"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.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">b</span><span class="p">,</span> <span class="n">nested</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="c1"># type: (bytes, bool) -&gt; None</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">b</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">)</span>
<span class="k">if</span> <span class="n">nested</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write_var_int64</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">b</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">b</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">byte_count</span> <span class="o">+=</span> <span class="nb">len</span><span class="p">(</span><span class="n">b</span><span class="p">)</span></div>
<div class="viewcode-block" id="OutputStream.write_byte"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.write_byte">[docs]</a> <span class="k">def</span> <span class="nf">write_byte</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">val</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="nb">chr</span><span class="p">(</span><span class="n">val</span><span class="p">)</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="s1">&#39;latin-1&#39;</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">byte_count</span> <span class="o">+=</span> <span class="mi">1</span></div>
<div class="viewcode-block" id="OutputStream.write_var_int64"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.write_var_int64">[docs]</a> <span class="k">def</span> <span class="nf">write_var_int64</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">v</span><span class="p">):</span>
<span class="c1"># type: (int) -&gt; None</span>
<span class="k">if</span> <span class="n">v</span> <span class="o">&lt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">v</span> <span class="o">+=</span> <span class="mi">1</span> <span class="o">&lt;&lt;</span> <span class="mi">64</span>
<span class="k">if</span> <span class="n">v</span> <span class="o">&lt;=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Value too large (negative).&#39;</span><span class="p">)</span>
<span class="k">while</span> <span class="kc">True</span><span class="p">:</span>
<span class="n">bits</span> <span class="o">=</span> <span class="n">v</span> <span class="o">&amp;</span> <span class="mh">0x7F</span>
<span class="n">v</span> <span class="o">&gt;&gt;=</span> <span class="mi">7</span>
<span class="k">if</span> <span class="n">v</span><span class="p">:</span>
<span class="n">bits</span> <span class="o">|=</span> <span class="mh">0x80</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write_byte</span><span class="p">(</span><span class="n">bits</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">v</span><span class="p">:</span>
<span class="k">break</span></div>
<div class="viewcode-block" id="OutputStream.write_bigendian_int64"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.write_bigendian_int64">[docs]</a> <span class="k">def</span> <span class="nf">write_bigendian_int64</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">v</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="n">struct</span><span class="o">.</span><span class="n">pack</span><span class="p">(</span><span class="s1">&#39;&gt;q&#39;</span><span class="p">,</span> <span class="n">v</span><span class="p">))</span></div>
<div class="viewcode-block" id="OutputStream.write_bigendian_uint64"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.write_bigendian_uint64">[docs]</a> <span class="k">def</span> <span class="nf">write_bigendian_uint64</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">v</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="n">struct</span><span class="o">.</span><span class="n">pack</span><span class="p">(</span><span class="s1">&#39;&gt;Q&#39;</span><span class="p">,</span> <span class="n">v</span><span class="p">))</span></div>
<div class="viewcode-block" id="OutputStream.write_bigendian_int32"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.write_bigendian_int32">[docs]</a> <span class="k">def</span> <span class="nf">write_bigendian_int32</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">v</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="n">struct</span><span class="o">.</span><span class="n">pack</span><span class="p">(</span><span class="s1">&#39;&gt;i&#39;</span><span class="p">,</span> <span class="n">v</span><span class="p">))</span></div>
<div class="viewcode-block" id="OutputStream.write_bigendian_double"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.write_bigendian_double">[docs]</a> <span class="k">def</span> <span class="nf">write_bigendian_double</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">v</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="n">struct</span><span class="o">.</span><span class="n">pack</span><span class="p">(</span><span class="s1">&#39;&gt;d&#39;</span><span class="p">,</span> <span class="n">v</span><span class="p">))</span></div>
<div class="viewcode-block" id="OutputStream.get"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.get">[docs]</a> <span class="k">def</span> <span class="nf">get</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; bytes</span>
<span class="k">return</span> <span class="sa">b</span><span class="s1">&#39;&#39;</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">data</span><span class="p">)</span></div>
<div class="viewcode-block" id="OutputStream.size"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.OutputStream.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="c1"># type: () -&gt; int</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">byte_count</span></div>
<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="c1"># type: () -&gt; None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data</span> <span class="o">=</span> <span class="p">[]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">byte_count</span> <span class="o">=</span> <span class="mi">0</span></div>
<div class="viewcode-block" id="ByteCountingOutputStream"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.ByteCountingOutputStream">[docs]</a><span class="k">class</span> <span class="nc">ByteCountingOutputStream</span><span class="p">(</span><span class="n">OutputStream</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> A pure Python implementation of stream.ByteCountingOutputStream.&quot;&quot;&quot;</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="c1"># Note that we don&#39;t actually use any of the data initialized by our super.</span>
<span class="nb">super</span><span class="p">(</span><span class="n">ByteCountingOutputStream</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="fm">__init__</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">count</span> <span class="o">=</span> <span class="mi">0</span>
<div class="viewcode-block" id="ByteCountingOutputStream.write"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.ByteCountingOutputStream.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">byte_array</span><span class="p">,</span> <span class="n">nested</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="c1"># type: (bytes, bool) -&gt; None</span>
<span class="n">blen</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">byte_array</span><span class="p">)</span>
<span class="k">if</span> <span class="n">nested</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">write_var_int64</span><span class="p">(</span><span class="n">blen</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">count</span> <span class="o">+=</span> <span class="n">blen</span></div>
<div class="viewcode-block" id="ByteCountingOutputStream.write_byte"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.ByteCountingOutputStream.write_byte">[docs]</a> <span class="k">def</span> <span class="nf">write_byte</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">_</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">count</span> <span class="o">+=</span> <span class="mi">1</span></div>
<div class="viewcode-block" id="ByteCountingOutputStream.get_count"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.ByteCountingOutputStream.get_count">[docs]</a> <span class="k">def</span> <span class="nf">get_count</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">count</span></div>
<div class="viewcode-block" id="ByteCountingOutputStream.get"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.ByteCountingOutputStream.get">[docs]</a> <span class="k">def</span> <span class="nf">get</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<span class="k">def</span> <span class="fm">__str__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;&lt;</span><span class="si">%s</span><span class="s1"> </span><span class="si">%s</span><span class="s1">&gt;&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">count</span><span class="p">)</span></div>
<div class="viewcode-block" id="InputStream"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream">[docs]</a><span class="k">class</span> <span class="nc">InputStream</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> A pure Python implementation of stream.InputStream.&quot;&quot;&quot;</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">data</span><span class="p">):</span>
<span class="c1"># type: (bytes) -&gt; None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">data</span> <span class="o">=</span> <span class="n">data</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pos</span> <span class="o">=</span> <span class="mi">0</span>
<span class="c1"># The behavior of looping over a byte-string and obtaining byte characters</span>
<span class="c1"># has been changed between python 2 and 3.</span>
<span class="c1"># b = b&#39;\xff\x01&#39;</span>
<span class="c1"># Python 2:</span>
<span class="c1"># b[0] = &#39;\xff&#39;</span>
<span class="c1"># ord(b[0]) = 255</span>
<span class="c1"># Python 3:</span>
<span class="c1"># b[0] = 255</span>
<span class="k">if</span> <span class="n">sys</span><span class="o">.</span><span class="n">version_info</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">&gt;=</span> <span class="mi">3</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">read_byte</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">read_byte_py3</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">read_byte</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">read_byte_py2</span>
<div class="viewcode-block" id="InputStream.size"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.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="k">return</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">data</span><span class="p">)</span> <span class="o">-</span> <span class="bp">self</span><span class="o">.</span><span class="n">pos</span></div>
<div class="viewcode-block" id="InputStream.read"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.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">size</span><span class="p">):</span>
<span class="c1"># type: (int) -&gt; bytes</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pos</span> <span class="o">+=</span> <span class="n">size</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">pos</span> <span class="o">-</span> <span class="n">size</span> <span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">pos</span><span class="p">]</span></div>
<div class="viewcode-block" id="InputStream.read_all"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.read_all">[docs]</a> <span class="k">def</span> <span class="nf">read_all</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">nested</span><span class="p">):</span>
<span class="c1"># type: (bool) -&gt; bytes</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">read_var_int64</span><span class="p">()</span> <span class="k">if</span> <span class="n">nested</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">size</span><span class="p">())</span></div>
<div class="viewcode-block" id="InputStream.read_byte_py2"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.read_byte_py2">[docs]</a> <span class="k">def</span> <span class="nf">read_byte_py2</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; int</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pos</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="c1"># mypy tests against python 3.x, where this is an error:</span>
<span class="k">return</span> <span class="nb">ord</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">pos</span> <span class="o">-</span> <span class="mi">1</span><span class="p">])</span> <span class="c1"># type: ignore[arg-type]</span></div>
<div class="viewcode-block" id="InputStream.read_byte_py3"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.read_byte_py3">[docs]</a> <span class="k">def</span> <span class="nf">read_byte_py3</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># type: () -&gt; int</span>
<span class="bp">self</span><span class="o">.</span><span class="n">pos</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">pos</span> <span class="o">-</span> <span class="mi">1</span><span class="p">]</span></div>
<div class="viewcode-block" id="InputStream.read_var_int64"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.read_var_int64">[docs]</a> <span class="k">def</span> <span class="nf">read_var_int64</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">shift</span> <span class="o">=</span> <span class="mi">0</span>
<span class="n">result</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">while</span> <span class="kc">True</span><span class="p">:</span>
<span class="n">byte</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">read_byte</span><span class="p">()</span>
<span class="k">if</span> <span class="n">byte</span> <span class="o">&lt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span><span class="s1">&#39;VarLong not terminated.&#39;</span><span class="p">)</span>
<span class="n">bits</span> <span class="o">=</span> <span class="n">byte</span> <span class="o">&amp;</span> <span class="mh">0x7F</span>
<span class="k">if</span> <span class="n">shift</span> <span class="o">&gt;=</span> <span class="mi">64</span> <span class="ow">or</span> <span class="p">(</span><span class="n">shift</span> <span class="o">&gt;=</span> <span class="mi">63</span> <span class="ow">and</span> <span class="n">bits</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span><span class="s1">&#39;VarLong too long.&#39;</span><span class="p">)</span>
<span class="n">result</span> <span class="o">|=</span> <span class="n">bits</span> <span class="o">&lt;&lt;</span> <span class="n">shift</span>
<span class="n">shift</span> <span class="o">+=</span> <span class="mi">7</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">byte</span> <span class="o">&amp;</span> <span class="mh">0x80</span><span class="p">:</span>
<span class="k">break</span>
<span class="k">if</span> <span class="n">result</span> <span class="o">&gt;=</span> <span class="mi">1</span> <span class="o">&lt;&lt;</span> <span class="mi">63</span><span class="p">:</span>
<span class="n">result</span> <span class="o">-=</span> <span class="mi">1</span> <span class="o">&lt;&lt;</span> <span class="mi">64</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="InputStream.read_bigendian_int64"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.read_bigendian_int64">[docs]</a> <span class="k">def</span> <span class="nf">read_bigendian_int64</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">struct</span><span class="o">.</span><span class="n">unpack</span><span class="p">(</span><span class="s1">&#39;&gt;q&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="mi">8</span><span class="p">))[</span><span class="mi">0</span><span class="p">]</span></div>
<div class="viewcode-block" id="InputStream.read_bigendian_uint64"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.read_bigendian_uint64">[docs]</a> <span class="k">def</span> <span class="nf">read_bigendian_uint64</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">struct</span><span class="o">.</span><span class="n">unpack</span><span class="p">(</span><span class="s1">&#39;&gt;Q&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="mi">8</span><span class="p">))[</span><span class="mi">0</span><span class="p">]</span></div>
<div class="viewcode-block" id="InputStream.read_bigendian_int32"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.read_bigendian_int32">[docs]</a> <span class="k">def</span> <span class="nf">read_bigendian_int32</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">struct</span><span class="o">.</span><span class="n">unpack</span><span class="p">(</span><span class="s1">&#39;&gt;i&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="mi">4</span><span class="p">))[</span><span class="mi">0</span><span class="p">]</span></div>
<div class="viewcode-block" id="InputStream.read_bigendian_double"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.InputStream.read_bigendian_double">[docs]</a> <span class="k">def</span> <span class="nf">read_bigendian_double</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">struct</span><span class="o">.</span><span class="n">unpack</span><span class="p">(</span><span class="s1">&#39;&gt;d&#39;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="mi">8</span><span class="p">))[</span><span class="mi">0</span><span class="p">]</span></div></div>
<div class="viewcode-block" id="get_varint_size"><a class="viewcode-back" href="../../../apache_beam.coders.slow_stream.html#apache_beam.coders.slow_stream.get_varint_size">[docs]</a><span class="k">def</span> <span class="nf">get_varint_size</span><span class="p">(</span><span class="n">v</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd"> Returns the size of the given integer value when encode as a VarInt.&quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">v</span> <span class="o">&lt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">v</span> <span class="o">+=</span> <span class="mi">1</span> <span class="o">&lt;&lt;</span> <span class="mi">64</span>
<span class="k">if</span> <span class="n">v</span> <span class="o">&lt;=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Value too large (negative).&#39;</span><span class="p">)</span>
<span class="n">varint_size</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">while</span> <span class="kc">True</span><span class="p">:</span>
<span class="n">varint_size</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="n">v</span> <span class="o">&gt;&gt;=</span> <span class="mi">7</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">v</span><span class="p">:</span>
<span class="k">break</span>
<span class="k">return</span> <span class="n">varint_size</span></div>
</pre></div>
</div>
<div class="articleComments">
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright .
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/snide/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
var DOCUMENTATION_OPTIONS = {
URL_ROOT:'../../../',
VERSION:'',
COLLAPSE_INDEX:false,
FILE_SUFFIX:'.html',
HAS_SOURCE: true,
SOURCELINK_SUFFIX: '.txt'
};
</script>
<script type="text/javascript" src="../../../_static/jquery.js"></script>
<script type="text/javascript" src="../../../_static/underscore.js"></script>
<script type="text/javascript" src="../../../_static/doctools.js"></script>
<script type="text/javascript" src="../../../_static/js/theme.js"></script>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.StickyNav.enable();
});
</script>
</body>
</html>