blob: acf457c0a42e6c6f516944404cb4a49effc39e27 [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.io.filesystemio &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.io.filesystemio</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.io.filesystemio</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;Utilities for ``FileSystem`` implementations.&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">__future__</span> <span class="k">import</span> <span class="n">absolute_import</span>
<span class="kn">import</span> <span class="nn">abc</span>
<span class="kn">import</span> <span class="nn">io</span>
<span class="kn">import</span> <span class="nn">os</span>
<span class="kn">from</span> <span class="nn">builtins</span> <span class="k">import</span> <span class="nb">object</span>
<span class="kn">from</span> <span class="nn">future.utils</span> <span class="k">import</span> <span class="n">with_metaclass</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s1">&#39;Downloader&#39;</span><span class="p">,</span> <span class="s1">&#39;Uploader&#39;</span><span class="p">,</span> <span class="s1">&#39;DownloaderStream&#39;</span><span class="p">,</span> <span class="s1">&#39;UploaderStream&#39;</span><span class="p">,</span>
<span class="s1">&#39;PipeStream&#39;</span><span class="p">]</span>
<div class="viewcode-block" id="Downloader"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.Downloader">[docs]</a><span class="k">class</span> <span class="nc">Downloader</span><span class="p">(</span><span class="n">with_metaclass</span><span class="p">(</span><span class="n">abc</span><span class="o">.</span><span class="n">ABCMeta</span><span class="p">,</span> <span class="nb">object</span><span class="p">)):</span>
<span class="sd">&quot;&quot;&quot;Download interface for a single file.</span>
<span class="sd"> Implementations should support random access reads.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nd">@abc</span><span class="o">.</span><span class="n">abstractproperty</span>
<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="sd">&quot;&quot;&quot;Size of file to download.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Downloader.get_range"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.Downloader.get_range">[docs]</a> <span class="nd">@abc</span><span class="o">.</span><span class="n">abstractmethod</span>
<span class="k">def</span> <span class="nf">get_range</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">start</span><span class="p">,</span> <span class="n">end</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Retrieve a given byte range [start, end) from this download.</span>
<span class="sd"> Range must be in this form:</span>
<span class="sd"> 0 &lt;= start &lt; end: Fetch the bytes from start to end.</span>
<span class="sd"> Args:</span>
<span class="sd"> start: (int) Initial byte offset.</span>
<span class="sd"> end: (int) Final byte offset, exclusive.</span>
<span class="sd"> Returns:</span>
<span class="sd"> (string) A buffer containing the requested data.</span>
<span class="sd"> &quot;&quot;&quot;</span></div></div>
<div class="viewcode-block" id="Uploader"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.Uploader">[docs]</a><span class="k">class</span> <span class="nc">Uploader</span><span class="p">(</span><span class="n">with_metaclass</span><span class="p">(</span><span class="n">abc</span><span class="o">.</span><span class="n">ABCMeta</span><span class="p">,</span> <span class="nb">object</span><span class="p">)):</span>
<span class="sd">&quot;&quot;&quot;Upload interface for a single file.&quot;&quot;&quot;</span>
<div class="viewcode-block" id="Uploader.put"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.Uploader.put">[docs]</a> <span class="nd">@abc</span><span class="o">.</span><span class="n">abstractmethod</span>
<span class="k">def</span> <span class="nf">put</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="sd">&quot;&quot;&quot;Write data to file sequentially.</span>
<span class="sd"> Args:</span>
<span class="sd"> data: (memoryview) Data to write.</span>
<span class="sd"> &quot;&quot;&quot;</span></div>
<div class="viewcode-block" id="Uploader.finish"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.Uploader.finish">[docs]</a> <span class="nd">@abc</span><span class="o">.</span><span class="n">abstractmethod</span>
<span class="k">def</span> <span class="nf">finish</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Signal to upload any remaining data and close the file.</span>
<span class="sd"> File should be fully written upon return from this method.</span>
<span class="sd"> Raises:</span>
<span class="sd"> Any error encountered during the upload.</span>
<span class="sd"> &quot;&quot;&quot;</span></div></div>
<div class="viewcode-block" id="DownloaderStream"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.DownloaderStream">[docs]</a><span class="k">class</span> <span class="nc">DownloaderStream</span><span class="p">(</span><span class="n">io</span><span class="o">.</span><span class="n">RawIOBase</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Provides a stream interface for Downloader objects.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">downloader</span><span class="p">,</span> <span class="n">mode</span><span class="o">=</span><span class="s1">&#39;r&#39;</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes the stream.</span>
<span class="sd"> Args:</span>
<span class="sd"> downloader: (Downloader) Filesystem dependent implementation.</span>
<span class="sd"> mode: (string) Python mode attribute for this stream.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_downloader</span> <span class="o">=</span> <span class="n">downloader</span>
<span class="bp">self</span><span class="o">.</span><span class="n">mode</span> <span class="o">=</span> <span class="n">mode</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">=</span> <span class="mi">0</span>
<div class="viewcode-block" id="DownloaderStream.readinto"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.DownloaderStream.readinto">[docs]</a> <span class="k">def</span> <span class="nf">readinto</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="sd">&quot;&quot;&quot;Read up to len(b) bytes into b.</span>
<span class="sd"> Returns number of bytes read (0 for EOF).</span>
<span class="sd"> Args:</span>
<span class="sd"> b: (bytearray/memoryview) Buffer to read into.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_checkClosed</span><span class="p">()</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_downloader</span><span class="o">.</span><span class="n">size</span><span class="p">:</span>
<span class="k">return</span> <span class="mi">0</span>
<span class="n">start</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_position</span>
<span class="n">end</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">+</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">_downloader</span><span class="o">.</span><span class="n">size</span><span class="p">)</span>
<span class="n">data</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_downloader</span><span class="o">.</span><span class="n">get_range</span><span class="p">(</span><span class="n">start</span><span class="p">,</span> <span class="n">end</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">+=</span> <span class="nb">len</span><span class="p">(</span><span class="n">data</span><span class="p">)</span>
<span class="n">b</span><span class="p">[:</span><span class="nb">len</span><span class="p">(</span><span class="n">data</span><span class="p">)]</span> <span class="o">=</span> <span class="n">data</span>
<span class="k">return</span> <span class="nb">len</span><span class="p">(</span><span class="n">data</span><span class="p">)</span></div>
<div class="viewcode-block" id="DownloaderStream.seek"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.DownloaderStream.seek">[docs]</a> <span class="k">def</span> <span class="nf">seek</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">offset</span><span class="p">,</span> <span class="n">whence</span><span class="o">=</span><span class="n">os</span><span class="o">.</span><span class="n">SEEK_SET</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Set the stream&#39;s current offset.</span>
<span class="sd"> Note if the new offset is out of bound, it is adjusted to either 0 or EOF.</span>
<span class="sd"> Args:</span>
<span class="sd"> offset: seek offset as number.</span>
<span class="sd"> whence: seek mode. Supported modes are os.SEEK_SET (absolute seek),</span>
<span class="sd"> os.SEEK_CUR (seek relative to the current position), and os.SEEK_END</span>
<span class="sd"> (seek relative to the end, offset should be negative).</span>
<span class="sd"> Raises:</span>
<span class="sd"> ``ValueError``: When this stream is closed or if whence is invalid.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_checkClosed</span><span class="p">()</span>
<span class="k">if</span> <span class="n">whence</span> <span class="o">==</span> <span class="n">os</span><span class="o">.</span><span class="n">SEEK_SET</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">=</span> <span class="n">offset</span>
<span class="k">elif</span> <span class="n">whence</span> <span class="o">==</span> <span class="n">os</span><span class="o">.</span><span class="n">SEEK_CUR</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">+=</span> <span class="n">offset</span>
<span class="k">elif</span> <span class="n">whence</span> <span class="o">==</span> <span class="n">os</span><span class="o">.</span><span class="n">SEEK_END</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_downloader</span><span class="o">.</span><span class="n">size</span> <span class="o">+</span> <span class="n">offset</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;Whence mode </span><span class="si">%r</span><span class="s1"> is invalid.&#39;</span> <span class="o">%</span> <span class="n">whence</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_position</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_downloader</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">_position</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_position</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_position</span></div>
<div class="viewcode-block" id="DownloaderStream.tell"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.DownloaderStream.tell">[docs]</a> <span class="k">def</span> <span class="nf">tell</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Tell the stream&#39;s current offset.</span>
<span class="sd"> Returns:</span>
<span class="sd"> current offset in reading this stream.</span>
<span class="sd"> Raises:</span>
<span class="sd"> ``ValueError``: When this stream is closed.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_checkClosed</span><span class="p">()</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_position</span></div>
<div class="viewcode-block" id="DownloaderStream.seekable"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.DownloaderStream.seekable">[docs]</a> <span class="k">def</span> <span class="nf">seekable</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span></div>
<div class="viewcode-block" id="DownloaderStream.readable"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.DownloaderStream.readable">[docs]</a> <span class="k">def</span> <span class="nf">readable</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span></div></div>
<div class="viewcode-block" id="UploaderStream"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.UploaderStream">[docs]</a><span class="k">class</span> <span class="nc">UploaderStream</span><span class="p">(</span><span class="n">io</span><span class="o">.</span><span class="n">RawIOBase</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Provides a stream interface for Uploader objects.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">uploader</span><span class="p">,</span> <span class="n">mode</span><span class="o">=</span><span class="s1">&#39;w&#39;</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes the stream.</span>
<span class="sd"> Args:</span>
<span class="sd"> uploader: (Uploader) Filesystem dependent implementation.</span>
<span class="sd"> mode: (string) Python mode attribute for this stream.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_uploader</span> <span class="o">=</span> <span class="n">uploader</span>
<span class="bp">self</span><span class="o">.</span><span class="n">mode</span> <span class="o">=</span> <span class="n">mode</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_position</span> <span class="o">=</span> <span class="mi">0</span>
<div class="viewcode-block" id="UploaderStream.tell"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.UploaderStream.tell">[docs]</a> <span class="k">def</span> <span class="nf">tell</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">_position</span></div>
<div class="viewcode-block" id="UploaderStream.write"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.UploaderStream.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="sd">&quot;&quot;&quot;Write bytes from b.</span>
<span class="sd"> Returns number of bytes written (&lt;= len(b)).</span>
<span class="sd"> Args:</span>
<span class="sd"> b: (memoryview) Buffer with data to write.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_checkClosed</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_uploader</span><span class="o">.</span><span class="n">put</span><span class="p">(</span><span class="n">b</span><span class="p">)</span>
<span class="n">bytes_written</span> <span class="o">=</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">_position</span> <span class="o">+=</span> <span class="n">bytes_written</span>
<span class="k">return</span> <span class="n">bytes_written</span></div>
<div class="viewcode-block" id="UploaderStream.close"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.UploaderStream.close">[docs]</a> <span class="k">def</span> <span class="nf">close</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Complete the upload and close this stream.</span>
<span class="sd"> This method has no effect if the stream is already closed.</span>
<span class="sd"> Raises:</span>
<span class="sd"> Any error encountered by the uploader.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">closed</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_uploader</span><span class="o">.</span><span class="n">finish</span><span class="p">()</span>
<span class="nb">super</span><span class="p">(</span><span class="n">UploaderStream</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="n">close</span><span class="p">()</span></div>
<div class="viewcode-block" id="UploaderStream.writable"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.UploaderStream.writable">[docs]</a> <span class="k">def</span> <span class="nf">writable</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">True</span></div></div>
<div class="viewcode-block" id="PipeStream"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.PipeStream">[docs]</a><span class="k">class</span> <span class="nc">PipeStream</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A class that presents a pipe connection as a readable stream.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">recv_pipe</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">conn</span> <span class="o">=</span> <span class="n">recv_pipe</span>
<span class="bp">self</span><span class="o">.</span><span class="n">closed</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">position</span> <span class="o">=</span> <span class="mi">0</span>
<span class="bp">self</span><span class="o">.</span><span class="n">remaining</span> <span class="o">=</span> <span class="sa">b</span><span class="s1">&#39;&#39;</span>
<div class="viewcode-block" id="PipeStream.read"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.PipeStream.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="sd">&quot;&quot;&quot;Read data from the wrapped pipe connection.</span>
<span class="sd"> Args:</span>
<span class="sd"> size: Number of bytes to read. Actual number of bytes read is always</span>
<span class="sd"> equal to size unless EOF is reached.</span>
<span class="sd"> Returns:</span>
<span class="sd"> data read as str.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">data_list</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">bytes_read</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">while</span> <span class="n">bytes_read</span> <span class="o">&lt;</span> <span class="n">size</span><span class="p">:</span>
<span class="n">bytes_from_remaining</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">size</span> <span class="o">-</span> <span class="n">bytes_read</span><span class="p">,</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">remaining</span><span class="p">))</span>
<span class="n">data_list</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">remaining</span><span class="p">[</span><span class="mi">0</span><span class="p">:</span><span class="n">bytes_from_remaining</span><span class="p">])</span>
<span class="bp">self</span><span class="o">.</span><span class="n">remaining</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">remaining</span><span class="p">[</span><span class="n">bytes_from_remaining</span><span class="p">:]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">position</span> <span class="o">+=</span> <span class="n">bytes_from_remaining</span>
<span class="n">bytes_read</span> <span class="o">+=</span> <span class="n">bytes_from_remaining</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">remaining</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">remaining</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">conn</span><span class="o">.</span><span class="n">recv_bytes</span><span class="p">()</span>
<span class="k">except</span> <span class="ne">EOFError</span><span class="p">:</span>
<span class="k">break</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="n">data_list</span><span class="p">)</span></div>
<div class="viewcode-block" id="PipeStream.tell"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.PipeStream.tell">[docs]</a> <span class="k">def</span> <span class="nf">tell</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Tell the file&#39;s current offset.</span>
<span class="sd"> Returns:</span>
<span class="sd"> current offset in reading this file.</span>
<span class="sd"> Raises:</span>
<span class="sd"> ``ValueError``: When this stream is closed.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_check_open</span><span class="p">()</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">position</span></div>
<div class="viewcode-block" id="PipeStream.seek"><a class="viewcode-back" href="../../../apache_beam.io.filesystemio.html#apache_beam.io.filesystemio.PipeStream.seek">[docs]</a> <span class="k">def</span> <span class="nf">seek</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">offset</span><span class="p">,</span> <span class="n">whence</span><span class="o">=</span><span class="n">os</span><span class="o">.</span><span class="n">SEEK_SET</span><span class="p">):</span>
<span class="c1"># The apitools library used by the gcsio.Uploader class insists on seeking</span>
<span class="c1"># to the end of a stream to do a check before completing an upload, so we</span>
<span class="c1"># must have this no-op method here in that case.</span>
<span class="k">if</span> <span class="n">whence</span> <span class="o">==</span> <span class="n">os</span><span class="o">.</span><span class="n">SEEK_END</span> <span class="ow">and</span> <span class="n">offset</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span>
<span class="k">elif</span> <span class="n">whence</span> <span class="o">==</span> <span class="n">os</span><span class="o">.</span><span class="n">SEEK_SET</span> <span class="ow">and</span> <span class="n">offset</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">position</span><span class="p">:</span>
<span class="k">return</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<span class="k">def</span> <span class="nf">_check_open</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">closed</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">IOError</span><span class="p">(</span><span class="s1">&#39;Stream is closed.&#39;</span><span class="p">)</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>