blob: 863606a8934faa5bdcb4f902ef47d73326d5ec41 [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.runners.dataflow.native_io.iobase &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.runners.dataflow.native_io.iobase</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.dataflow.native_io.iobase</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;Dataflow native sources and sinks.</span>
<span class="sd">For internal use only; no backwards-compatibility guarantees.</span>
<span class="sd">&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">logging</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">apache_beam</span> <span class="k">import</span> <span class="n">pvalue</span>
<span class="kn">from</span> <span class="nn">apache_beam.io</span> <span class="k">import</span> <span class="n">iobase</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="k">import</span> <span class="n">ptransform</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.display</span> <span class="k">import</span> <span class="n">HasDisplayData</span>
<span class="k">def</span> <span class="nf">_dict_printable_fields</span><span class="p">(</span><span class="n">dict_object</span><span class="p">,</span> <span class="n">skip_fields</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a list of strings for the interesting fields of a dict.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="p">[</span><span class="s1">&#39;</span><span class="si">%s</span><span class="s1">=</span><span class="si">%r</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">value</span><span class="p">)</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">dict_object</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="c1"># want to output value 0 but not None nor []</span>
<span class="k">if</span> <span class="p">(</span><span class="n">value</span> <span class="ow">or</span> <span class="n">value</span> <span class="o">==</span> <span class="mi">0</span><span class="p">)</span>
<span class="ow">and</span> <span class="n">name</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">skip_fields</span><span class="p">]</span>
<span class="n">_minor_fields</span> <span class="o">=</span> <span class="p">[</span><span class="s1">&#39;coder&#39;</span><span class="p">,</span> <span class="s1">&#39;key_coder&#39;</span><span class="p">,</span> <span class="s1">&#39;value_coder&#39;</span><span class="p">,</span>
<span class="s1">&#39;config_bytes&#39;</span><span class="p">,</span> <span class="s1">&#39;elements&#39;</span><span class="p">,</span>
<span class="s1">&#39;append_trailing_newlines&#39;</span><span class="p">,</span> <span class="s1">&#39;strip_trailing_newlines&#39;</span><span class="p">,</span>
<span class="s1">&#39;compression_type&#39;</span><span class="p">]</span>
<div class="viewcode-block" id="NativeSource"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSource">[docs]</a><span class="k">class</span> <span class="nc">NativeSource</span><span class="p">(</span><span class="n">iobase</span><span class="o">.</span><span class="n">SourceBase</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A source implemented by Dataflow service.</span>
<span class="sd"> This class is to be only inherited by sources natively implemented by Cloud</span>
<span class="sd"> Dataflow service, hence should not be sub-classed by users.</span>
<span class="sd"> This class is deprecated and should not be used to define new sources.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="NativeSource.reader"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSource.reader">[docs]</a> <span class="k">def</span> <span class="nf">reader</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a NativeSourceReader instance associated with this source.&quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<div class="viewcode-block" id="NativeSource.is_bounded"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSource.is_bounded">[docs]</a> <span class="k">def</span> <span class="nf">is_bounded</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>
<span class="k">def</span> <span class="nf">__repr__</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">{name}</span><span class="s1"> </span><span class="si">{vals}</span><span class="s1">&gt;&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">name</span><span class="o">=</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="n">vals</span><span class="o">=</span><span class="s1">&#39;, &#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">_dict_printable_fields</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="vm">__dict__</span><span class="p">,</span>
<span class="n">_minor_fields</span><span class="p">)))</span></div>
<div class="viewcode-block" id="NativeSourceReader"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSourceReader">[docs]</a><span class="k">class</span> <span class="nc">NativeSourceReader</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A reader for a source implemented by Dataflow service.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__enter__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Opens everything necessary for a reader to function properly.&quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span>
<span class="k">def</span> <span class="nf">__exit__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">exception_type</span><span class="p">,</span> <span class="n">exception_value</span><span class="p">,</span> <span class="n">traceback</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Cleans up after a reader executed.&quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span>
<span class="k">def</span> <span class="nf">__iter__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns an iterator over all the records of the source.&quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">returns_windowed_values</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns whether this reader returns windowed values.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="kc">False</span>
<div class="viewcode-block" id="NativeSourceReader.get_progress"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSourceReader.get_progress">[docs]</a> <span class="k">def</span> <span class="nf">get_progress</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a representation of how far the reader has read.</span>
<span class="sd"> Returns:</span>
<span class="sd"> A SourceReaderProgress object that gives the current progress of the</span>
<span class="sd"> reader.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span></div>
<div class="viewcode-block" id="NativeSourceReader.request_dynamic_split"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSourceReader.request_dynamic_split">[docs]</a> <span class="k">def</span> <span class="nf">request_dynamic_split</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">dynamic_split_request</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Attempts to split the input in two parts.</span>
<span class="sd"> The two parts are named the &quot;primary&quot; part and the &quot;residual&quot; part. The</span>
<span class="sd"> current &#39;NativeSourceReader&#39; keeps processing the primary part, while the</span>
<span class="sd"> residual part will be processed elsewhere (e.g. perhaps on a different</span>
<span class="sd"> worker).</span>
<span class="sd"> The primary and residual parts, if concatenated, must represent the</span>
<span class="sd"> same input as the current input of this &#39;NativeSourceReader&#39; before this</span>
<span class="sd"> call.</span>
<span class="sd"> The boundary between the primary part and the residual part is</span>
<span class="sd"> specified in a framework-specific way using &#39;DynamicSplitRequest&#39; e.g.,</span>
<span class="sd"> if the framework supports the notion of positions, it might be a</span>
<span class="sd"> position at which the input is asked to split itself (which is not</span>
<span class="sd"> necessarily the same position at which it *will* split itself); it</span>
<span class="sd"> might be an approximate fraction of input, or something else.</span>
<span class="sd"> This function returns a &#39;DynamicSplitResult&#39;, which encodes, in a</span>
<span class="sd"> framework-specific way, the information sufficient to construct a</span>
<span class="sd"> description of the resulting primary and residual inputs. For example, it</span>
<span class="sd"> might, again, be a position demarcating these parts, or it might be a pair</span>
<span class="sd"> of fully-specified input descriptions, or something else.</span>
<span class="sd"> After a successful call to &#39;request_dynamic_split()&#39;, subsequent calls</span>
<span class="sd"> should be interpreted relative to the new primary.</span>
<span class="sd"> Args:</span>
<span class="sd"> dynamic_split_request: A &#39;DynamicSplitRequest&#39; describing the split</span>
<span class="sd"> request.</span>
<span class="sd"> Returns:</span>
<span class="sd"> &#39;None&#39; if the &#39;DynamicSplitRequest&#39; cannot be honored (in that</span>
<span class="sd"> case the input represented by this &#39;NativeSourceReader&#39; stays the same),</span>
<span class="sd"> or a &#39;DynamicSplitResult&#39; describing how the input was split into a</span>
<span class="sd"> primary and residual part.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">logging</span><span class="o">.</span><span class="n">debug</span><span class="p">(</span>
<span class="s1">&#39;SourceReader </span><span class="si">%r</span><span class="s1"> does not support dynamic splitting. Ignoring dynamic &#39;</span>
<span class="s1">&#39;split request: </span><span class="si">%r</span><span class="s1">&#39;</span><span class="p">,</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">dynamic_split_request</span><span class="p">)</span>
<span class="k">return</span></div></div>
<div class="viewcode-block" id="ReaderProgress"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.ReaderProgress">[docs]</a><span class="k">class</span> <span class="nc">ReaderProgress</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A representation of how far a NativeSourceReader has read.&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">position</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">percent_complete</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">remaining_time</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">consumed_split_points</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">remaining_split_points</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">_position</span> <span class="o">=</span> <span class="n">position</span>
<span class="k">if</span> <span class="n">percent_complete</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">percent_complete</span> <span class="o">=</span> <span class="nb">float</span><span class="p">(</span><span class="n">percent_complete</span><span class="p">)</span>
<span class="k">if</span> <span class="n">percent_complete</span> <span class="o">&lt;</span> <span class="mi">0</span> <span class="ow">or</span> <span class="n">percent_complete</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;The percent_complete argument was </span><span class="si">%f</span><span class="s1">. Must be in range [0, 1].&#39;</span>
<span class="o">%</span> <span class="n">percent_complete</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_percent_complete</span> <span class="o">=</span> <span class="n">percent_complete</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_remaining_time</span> <span class="o">=</span> <span class="n">remaining_time</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_consumed_split_points</span> <span class="o">=</span> <span class="n">consumed_split_points</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_remaining_split_points</span> <span class="o">=</span> <span class="n">remaining_split_points</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">position</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns progress, represented as a ReaderPosition object.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_position</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">percent_complete</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns progress, represented as a percentage of total work.</span>
<span class="sd"> Progress range from 0.0 (beginning, nothing complete) to 1.0 (end of the</span>
<span class="sd"> work range, entire WorkItem complete).</span>
<span class="sd"> Returns:</span>
<span class="sd"> Progress represented as a percentage of total work.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_percent_complete</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">remaining_time</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns progress, represented as an estimated time remaining.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_remaining_time</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">consumed_split_points</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">_consumed_split_points</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">remaining_split_points</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">_remaining_split_points</span></div>
<div class="viewcode-block" id="ReaderPosition"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.ReaderPosition">[docs]</a><span class="k">class</span> <span class="nc">ReaderPosition</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A representation of position in an iteration of a &#39;NativeSourceReader&#39;.&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">end</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">byte_offset</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">record_index</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">shuffle_position</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">concat_position</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes ReaderPosition.</span>
<span class="sd"> A ReaderPosition may get instantiated for one of these position types. Only</span>
<span class="sd"> one of these should be specified.</span>
<span class="sd"> Args:</span>
<span class="sd"> end: position is past all other positions. For example, this may be used</span>
<span class="sd"> to represent the end position of an unbounded range.</span>
<span class="sd"> key: position is a string key.</span>
<span class="sd"> byte_offset: position is a byte offset.</span>
<span class="sd"> record_index: position is a record index</span>
<span class="sd"> shuffle_position: position is a base64 encoded shuffle position.</span>
<span class="sd"> concat_position: position is a &#39;ConcatPosition&#39;.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">end</span> <span class="o">=</span> <span class="n">end</span>
<span class="bp">self</span><span class="o">.</span><span class="n">key</span> <span class="o">=</span> <span class="n">key</span>
<span class="bp">self</span><span class="o">.</span><span class="n">byte_offset</span> <span class="o">=</span> <span class="n">byte_offset</span>
<span class="bp">self</span><span class="o">.</span><span class="n">record_index</span> <span class="o">=</span> <span class="n">record_index</span>
<span class="bp">self</span><span class="o">.</span><span class="n">shuffle_position</span> <span class="o">=</span> <span class="n">shuffle_position</span>
<span class="k">if</span> <span class="n">concat_position</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">concat_position</span><span class="p">,</span> <span class="n">ConcatPosition</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">concat_position</span> <span class="o">=</span> <span class="n">concat_position</span></div>
<div class="viewcode-block" id="ConcatPosition"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.ConcatPosition">[docs]</a><span class="k">class</span> <span class="nc">ConcatPosition</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A position that encapsulate an inner position and an index.</span>
<span class="sd"> This is used to represent the position of a source that encapsulate several</span>
<span class="sd"> other sources.</span>
<span class="sd"> &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">index</span><span class="p">,</span> <span class="n">position</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes ConcatPosition.</span>
<span class="sd"> Args:</span>
<span class="sd"> index: index of the source currently being read.</span>
<span class="sd"> position: inner position within the source currently being read.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">position</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">position</span><span class="p">,</span> <span class="n">ReaderPosition</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">index</span> <span class="o">=</span> <span class="n">index</span>
<span class="bp">self</span><span class="o">.</span><span class="n">position</span> <span class="o">=</span> <span class="n">position</span></div>
<div class="viewcode-block" id="DynamicSplitRequest"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.DynamicSplitRequest">[docs]</a><span class="k">class</span> <span class="nc">DynamicSplitRequest</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Specifies how &#39;NativeSourceReader.request_dynamic_split&#39; should split.</span>
<span class="sd"> &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">progress</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">progress</span><span class="p">,</span> <span class="n">ReaderProgress</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">progress</span> <span class="o">=</span> <span class="n">progress</span></div>
<div class="viewcode-block" id="DynamicSplitResult"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.DynamicSplitResult">[docs]</a><span class="k">class</span> <span class="nc">DynamicSplitResult</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="DynamicSplitResultWithPosition"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.DynamicSplitResultWithPosition">[docs]</a><span class="k">class</span> <span class="nc">DynamicSplitResultWithPosition</span><span class="p">(</span><span class="n">DynamicSplitResult</span><span class="p">):</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">stop_position</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">stop_position</span><span class="p">,</span> <span class="n">ReaderPosition</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">stop_position</span> <span class="o">=</span> <span class="n">stop_position</span></div>
<div class="viewcode-block" id="NativeSink"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSink">[docs]</a><span class="k">class</span> <span class="nc">NativeSink</span><span class="p">(</span><span class="n">HasDisplayData</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A sink implemented by Dataflow service.</span>
<span class="sd"> This class is to be only inherited by sinks natively implemented by Cloud</span>
<span class="sd"> Dataflow service, hence should not be sub-classed by users.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<div class="viewcode-block" id="NativeSink.writer"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSink.writer">[docs]</a> <span class="k">def</span> <span class="nf">writer</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns a SinkWriter for this source.&quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div>
<span class="k">def</span> <span class="nf">__repr__</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">{name}</span><span class="s1"> </span><span class="si">{vals}</span><span class="s1">&gt;&#39;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">name</span><span class="o">=</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="n">vals</span><span class="o">=</span><span class="n">_dict_printable_fields</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="vm">__dict__</span><span class="p">,</span> <span class="n">_minor_fields</span><span class="p">))</span></div>
<div class="viewcode-block" id="NativeSinkWriter"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSinkWriter">[docs]</a><span class="k">class</span> <span class="nc">NativeSinkWriter</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A writer for a sink implemented by Dataflow service.&quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">__enter__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Opens everything necessary for a writer to function properly.&quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span>
<span class="k">def</span> <span class="nf">__exit__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">exception_type</span><span class="p">,</span> <span class="n">exception_value</span><span class="p">,</span> <span class="n">traceback</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Cleans up after a writer executed.&quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">takes_windowed_values</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Returns whether this writer takes windowed values.&quot;&quot;&quot;</span>
<span class="k">return</span> <span class="kc">False</span>
<div class="viewcode-block" id="NativeSinkWriter.Write"><a class="viewcode-back" href="../../../../../apache_beam.runners.dataflow.native_io.iobase.html#apache_beam.runners.dataflow.native_io.iobase.NativeSinkWriter.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">o</span><span class="p">):</span> <span class="c1"># pylint: disable=invalid-name</span>
<span class="sd">&quot;&quot;&quot;Writes a record to the sink associated with this writer.&quot;&quot;&quot;</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span></div></div>
<span class="k">class</span> <span class="nc">_NativeWrite</span><span class="p">(</span><span class="n">ptransform</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A PTransform for writing to a Dataflow native sink.</span>
<span class="sd"> These are sinks that are implemented natively by the Dataflow service</span>
<span class="sd"> and hence should not be updated by users. These sinks are processed</span>
<span class="sd"> using a Dataflow native write transform.</span>
<span class="sd"> Applying this transform results in a ``pvalue.PDone``.</span>
<span class="sd"> &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">sink</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;Initializes a Write transform.</span>
<span class="sd"> Args:</span>
<span class="sd"> sink: Sink to use for the write</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="nb">super</span><span class="p">(</span><span class="n">_NativeWrite</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">sink</span> <span class="o">=</span> <span class="n">sink</span>
<span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_check_pcollection</span><span class="p">(</span><span class="n">pcoll</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">PDone</span><span class="p">(</span><span class="n">pcoll</span><span class="o">.</span><span class="n">pipeline</span><span class="p">)</span>
</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>