| |
| |
| <!DOCTYPE html> |
| <html class="writer-html5" lang="en" data-content_root="../../../"> |
| <head> |
| <meta charset="utf-8" /> |
| <meta name="viewport" content="width=device-width, initial-scale=1.0" /> |
| <title>apache_beam.io.textio — Apache Beam 2.67.0 documentation</title> |
| <link rel="stylesheet" type="text/css" href="../../../_static/pygments.css?v=b86133f3" /> |
| <link rel="stylesheet" type="text/css" href="../../../_static/css/theme.css?v=e59714d7" /> |
| |
| |
| <script src="../../../_static/jquery.js?v=5d32c60e"></script> |
| <script src="../../../_static/_sphinx_javascript_frameworks_compat.js?v=2cd50e6c"></script> |
| <script src="../../../_static/documentation_options.js?v=959b4fbe"></script> |
| <script src="../../../_static/doctools.js?v=9a2dae69"></script> |
| <script src="../../../_static/sphinx_highlight.js?v=dc90522c"></script> |
| <script src="../../../_static/js/theme.js"></script> |
| <link rel="index" title="Index" href="../../../genindex.html" /> |
| <link rel="search" title="Search" href="../../../search.html" /> |
| </head> |
| |
| <body class="wy-body-for-nav"> |
| <div class="wy-grid-for-nav"> |
| <nav data-toggle="wy-nav-shift" class="wy-nav-side"> |
| <div class="wy-side-scroll"> |
| <div class="wy-side-nav-search" > |
| |
| |
| |
| <a href="../../../index.html" class="icon icon-home"> |
| Apache Beam |
| </a> |
| <div role="search"> |
| <form id="rtd-search-form" class="wy-form" action="../../../search.html" method="get"> |
| <input type="text" name="q" placeholder="Search docs" aria-label="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="Navigation menu"> |
| <ul> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.coders.html">apache_beam.coders package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.dataframe.html">apache_beam.dataframe package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.io.html">apache_beam.io package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.metrics.html">apache_beam.metrics package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.ml.html">apache_beam.ml package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.options.html">apache_beam.options package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.portability.html">apache_beam.portability package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.runners.html">apache_beam.runners package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.testing.html">apache_beam.testing package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.transforms.html">apache_beam.transforms package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.typehints.html">apache_beam.typehints package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.utils.html">apache_beam.utils package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.yaml.html">apache_beam.yaml package</a></li> |
| </ul> |
| <ul> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.error.html">apache_beam.error module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li> |
| </ul> |
| |
| </div> |
| </div> |
| </nav> |
| |
| <section data-toggle="wy-nav-shift" class="wy-nav-content-wrap"><nav class="wy-nav-top" aria-label="Mobile navigation menu" > |
| <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="Page navigation"> |
| <ul class="wy-breadcrumbs"> |
| <li><a href="../../../index.html" class="icon icon-home" aria-label="Home"></a></li> |
| <li class="breadcrumb-item"><a href="../../index.html">Module code</a></li> |
| <li class="breadcrumb-item active">apache_beam.io.textio</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.textio</h1><div class="highlight"><pre> |
| <span></span><span class="c1">#</span> |
| <span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span> |
| <span class="c1"># contributor license agreements. See the NOTICE file distributed with</span> |
| <span class="c1"># this work for additional information regarding copyright ownership.</span> |
| <span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span> |
| <span class="c1"># (the "License"); you may not use this file except in compliance with</span> |
| <span class="c1"># the License. You may obtain a copy of the License at</span> |
| <span class="c1">#</span> |
| <span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span> |
| <span class="c1">#</span> |
| <span class="c1"># Unless required by applicable law or agreed to in writing, software</span> |
| <span class="c1"># distributed under the License is distributed on an "AS IS" BASIS,</span> |
| <span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span> |
| <span class="c1"># See the License for the specific language governing permissions and</span> |
| <span class="c1"># limitations under the License.</span> |
| <span class="c1">#</span> |
| |
| <span class="sd">"""A source and a sink for reading from and writing to text files."""</span> |
| |
| <span class="c1"># pytype: skip-file</span> |
| |
| <span class="kn">import</span><span class="w"> </span><span class="nn">logging</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">os</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">functools</span><span class="w"> </span><span class="kn">import</span> <span class="n">partial</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">typing</span><span class="w"> </span><span class="kn">import</span> <span class="n">TYPE_CHECKING</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">typing</span><span class="w"> </span><span class="kn">import</span> <span class="n">Any</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">typing</span><span class="w"> </span><span class="kn">import</span> <span class="n">Dict</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">typing</span><span class="w"> </span><span class="kn">import</span> <span class="n">Optional</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">typing</span><span class="w"> </span><span class="kn">import</span> <span class="n">Union</span> |
| |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam</span><span class="w"> </span><span class="kn">import</span> <span class="n">typehints</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.coders</span><span class="w"> </span><span class="kn">import</span> <span class="n">coders</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io</span><span class="w"> </span><span class="kn">import</span> <span class="n">filebasedsink</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io</span><span class="w"> </span><span class="kn">import</span> <span class="n">filebasedsource</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io</span><span class="w"> </span><span class="kn">import</span> <span class="n">iobase</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io.filebasedsource</span><span class="w"> </span><span class="kn">import</span> <span class="n">ReadAllFiles</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io.filesystem</span><span class="w"> </span><span class="kn">import</span> <span class="n">CompressionTypes</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io.iobase</span><span class="w"> </span><span class="kn">import</span> <span class="n">Read</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io.iobase</span><span class="w"> </span><span class="kn">import</span> <span class="n">Write</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.transforms</span><span class="w"> </span><span class="kn">import</span> <span class="n">PTransform</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.transforms.display</span><span class="w"> </span><span class="kn">import</span> <span class="n">DisplayDataItem</span> |
| |
| <span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io</span><span class="w"> </span><span class="kn">import</span> <span class="n">fileio</span> |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'ReadFromText'</span><span class="p">,</span> |
| <span class="s1">'ReadFromTextWithFilename'</span><span class="p">,</span> |
| <span class="s1">'ReadAllFromText'</span><span class="p">,</span> |
| <span class="s1">'ReadAllFromTextContinuously'</span><span class="p">,</span> |
| <span class="s1">'WriteToText'</span><span class="p">,</span> |
| <span class="s1">'ReadFromCsv'</span><span class="p">,</span> |
| <span class="s1">'WriteToCsv'</span><span class="p">,</span> |
| <span class="s1">'ReadFromJson'</span><span class="p">,</span> |
| <span class="s1">'WriteToJson'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="n">_LOGGER</span> <span class="o">=</span> <span class="n">logging</span><span class="o">.</span><span class="n">getLogger</span><span class="p">(</span><span class="vm">__name__</span><span class="p">)</span> |
| |
| |
| <span class="k">class</span><span class="w"> </span><span class="nc">_TextSource</span><span class="p">(</span><span class="n">filebasedsource</span><span class="o">.</span><span class="n">FileBasedSource</span><span class="p">):</span> |
| <span class="w"> </span><span class="sa">r</span><span class="sd">"""A source for reading text files.</span> |
| |
| <span class="sd"> Parses a text file as newline-delimited elements. Supports newline delimiters</span> |
| <span class="sd"> '\n' and '\r\n.</span> |
| |
| <span class="sd"> This implementation reads encoded text and uses the input coder's encoding to</span> |
| <span class="sd"> decode from bytes to str. This does not support ``UTF-16`` or ``UTF-32``</span> |
| <span class="sd"> encodings.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">DEFAULT_READ_BUFFER_SIZE</span> <span class="o">=</span> <span class="mi">8192</span> |
| |
| <span class="k">class</span><span class="w"> </span><span class="nc">ReadBuffer</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="c1"># A buffer that gives the buffered data and next position in the</span> |
| <span class="c1"># buffer that should be read.</span> |
| |
| <span class="k">def</span><span class="w"> </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="n">position</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">data</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="nd">@property</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">data</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">_data</span> |
| |
| <span class="nd">@data</span><span class="o">.</span><span class="n">setter</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">data</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="nb">bytes</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">value</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">position</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> |
| |
| <span class="nd">@position</span><span class="o">.</span><span class="n">setter</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">position</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">value</span> <span class="o">></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="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s1">'Cannot set position to </span><span class="si">%d</span><span class="s1"> since it</span><span class="se">\'</span><span class="s1">s larger than '</span> |
| <span class="s1">'size of data </span><span class="si">%d</span><span class="s1">.'</span> <span class="o">%</span> <span class="p">(</span><span class="n">value</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">_data</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">value</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">reset</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="sa">b</span><span class="s1">''</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="k">def</span><span class="w"> </span><span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">file_pattern</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="p">,</span> |
| <span class="n">strip_trailing_newlines</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">:</span> <span class="n">coders</span><span class="o">.</span><span class="n">Coder</span><span class="p">,</span> |
| <span class="n">buffer_size</span><span class="o">=</span><span class="n">DEFAULT_READ_BUFFER_SIZE</span><span class="p">,</span> |
| <span class="n">validate</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">skip_header_lines</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">header_processor_fns</span><span class="o">=</span><span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">),</span> |
| <span class="n">delimiter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">escapechar</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Initialize a _TextSource</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> header_processor_fns (tuple): a tuple of a `header_matcher` function</span> |
| <span class="sd"> and a `header_processor` function. The `header_matcher` should</span> |
| <span class="sd"> return `True` for all lines at the start of the file that are part</span> |
| <span class="sd"> of the file header and `False` otherwise. These header lines will</span> |
| <span class="sd"> not be yielded when reading records and instead passed into</span> |
| <span class="sd"> `header_processor` to be handled. If `skip_header_lines` and a</span> |
| <span class="sd"> `header_matcher` are both provided, the value of `skip_header_lines`</span> |
| <span class="sd"> lines will be skipped and the header will be processed from</span> |
| <span class="sd"> there.</span> |
| <span class="sd"> delimiter (bytes) Optional: delimiter to split records.</span> |
| <span class="sd"> Must not self-overlap, because self-overlapping delimiters cause</span> |
| <span class="sd"> ambiguous parsing.</span> |
| <span class="sd"> escapechar (bytes) Optional: a single byte to escape the records</span> |
| <span class="sd"> delimiter, can also escape itself.</span> |
| <span class="sd"> Raises:</span> |
| <span class="sd"> ValueError: if skip_lines is negative.</span> |
| |
| <span class="sd"> Please refer to documentation in class `ReadFromText` for the rest</span> |
| <span class="sd"> of the arguments.</span> |
| <span class="sd"> """</span> |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span> |
| <span class="n">file_pattern</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">compression_type</span><span class="p">,</span> |
| <span class="n">validate</span><span class="o">=</span><span class="n">validate</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_strip_trailing_newlines</span> <span class="o">=</span> <span class="n">strip_trailing_newlines</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_compression_type</span> <span class="o">=</span> <span class="n">compression_type</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span> <span class="o">=</span> <span class="n">coder</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_buffer_size</span> <span class="o">=</span> <span class="n">buffer_size</span> |
| <span class="k">if</span> <span class="n">skip_header_lines</span> <span class="o"><</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">'Cannot skip negative number of header lines: </span><span class="si">%d</span><span class="s1">'</span> <span class="o">%</span> <span class="n">skip_header_lines</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">skip_header_lines</span> <span class="o">></span> <span class="mi">10</span><span class="p">:</span> |
| <span class="n">_LOGGER</span><span class="o">.</span><span class="n">warning</span><span class="p">(</span> |
| <span class="s1">'Skipping </span><span class="si">%d</span><span class="s1"> header lines. Skipping large number of header '</span> |
| <span class="s1">'lines might significantly slow down processing.'</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_skip_header_lines</span> <span class="o">=</span> <span class="n">skip_header_lines</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_header_matcher</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_header_processor</span> <span class="o">=</span> <span class="n">header_processor_fns</span> |
| <span class="k">if</span> <span class="n">delimiter</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">delimiter</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">)</span> <span class="ow">or</span> <span class="nb">len</span><span class="p">(</span><span class="n">delimiter</span><span class="p">)</span> <span class="o">==</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">'Delimiter must be a non-empty bytes sequence.'</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_self_overlapping</span><span class="p">(</span><span class="n">delimiter</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">'Delimiter must not self-overlap.'</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_delimiter</span> <span class="o">=</span> <span class="n">delimiter</span> |
| <span class="k">if</span> <span class="n">escapechar</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">escapechar</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">escapechar</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"escapechar must be bytes of size 1: '</span><span class="si">%s</span><span class="s2">'"</span> <span class="o">%</span> <span class="n">escapechar</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_escapechar</span> <span class="o">=</span> <span class="n">escapechar</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="n">parent_dd</span> <span class="o">=</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">display_data</span><span class="p">()</span> |
| <span class="n">parent_dd</span><span class="p">[</span><span class="s1">'strip_newline'</span><span class="p">]</span> <span class="o">=</span> <span class="n">DisplayDataItem</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_strip_trailing_newlines</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">'Strip Trailing New Lines'</span><span class="p">)</span> |
| <span class="n">parent_dd</span><span class="p">[</span><span class="s1">'buffer_size'</span><span class="p">]</span> <span class="o">=</span> <span class="n">DisplayDataItem</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_buffer_size</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">'Buffer Size'</span><span class="p">)</span> |
| <span class="n">parent_dd</span><span class="p">[</span><span class="s1">'coder'</span><span class="p">]</span> <span class="o">=</span> <span class="n">DisplayDataItem</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="vm">__class__</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">'Coder'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">parent_dd</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">read_records</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_name</span><span class="p">,</span> <span class="n">range_tracker</span><span class="p">):</span> |
| <span class="n">start_offset</span> <span class="o">=</span> <span class="n">range_tracker</span><span class="o">.</span><span class="n">start_position</span><span class="p">()</span> |
| <span class="n">read_buffer</span> <span class="o">=</span> <span class="n">_TextSource</span><span class="o">.</span><span class="n">ReadBuffer</span><span class="p">(</span><span class="sa">b</span><span class="s1">''</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span> |
| |
| <span class="n">next_record_start_position</span> <span class="o">=</span> <span class="o">-</span><span class="mi">1</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">split_points_unclaimed</span><span class="p">(</span><span class="n">stop_position</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="mi">0</span> <span class="k">if</span> <span class="n">stop_position</span> <span class="o"><=</span> <span class="n">next_record_start_position</span> <span class="k">else</span> |
| <span class="n">iobase</span><span class="o">.</span><span class="n">RangeTracker</span><span class="o">.</span><span class="n">SPLIT_POINTS_UNKNOWN</span><span class="p">)</span> |
| |
| <span class="n">range_tracker</span><span class="o">.</span><span class="n">set_split_points_unclaimed_callback</span><span class="p">(</span><span class="n">split_points_unclaimed</span><span class="p">)</span> |
| |
| <span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">open_file</span><span class="p">(</span><span class="n">file_name</span><span class="p">)</span> <span class="k">as</span> <span class="n">file_to_read</span><span class="p">:</span> |
| <span class="n">position_after_processing_header_lines</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_process_header</span><span class="p">(</span><span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">))</span> |
| <span class="n">start_offset</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="n">start_offset</span><span class="p">,</span> <span class="n">position_after_processing_header_lines</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">start_offset</span> <span class="o">></span> <span class="n">position_after_processing_header_lines</span><span class="p">:</span> |
| <span class="c1"># Seeking to one delimiter length before the start index and ignoring</span> |
| <span class="c1"># the current line. If start_position is at beginning of the line, that</span> |
| <span class="c1"># line belongs to the current bundle, hence ignoring that is incorrect.</span> |
| <span class="c1"># Seeking to one delimiter before prevents that.</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_delimiter</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">start_offset</span> <span class="o">>=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_delimiter</span><span class="p">):</span> |
| <span class="n">required_position</span> <span class="o">=</span> <span class="n">start_offset</span> <span class="o">-</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_delimiter</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">required_position</span> <span class="o">=</span> <span class="n">start_offset</span> <span class="o">-</span> <span class="mi">1</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_escapechar</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># Need more bytes to check if the delimiter is escaped.</span> |
| <span class="c1"># Seek until the first escapechar if any.</span> |
| <span class="k">while</span> <span class="n">required_position</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">file_to_read</span><span class="o">.</span><span class="n">seek</span><span class="p">(</span><span class="n">required_position</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">file_to_read</span><span class="o">.</span><span class="n">read</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">_escapechar</span><span class="p">:</span> |
| <span class="n">required_position</span> <span class="o">-=</span> <span class="mi">1</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">break</span> |
| |
| <span class="n">file_to_read</span><span class="o">.</span><span class="n">seek</span><span class="p">(</span><span class="n">required_position</span><span class="p">)</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">reset</span><span class="p">()</span> |
| <span class="n">sep_bounds</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_find_separator_bounds</span><span class="p">(</span><span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">sep_bounds</span><span class="p">:</span> |
| <span class="c1"># Could not find a delimiter after required_position. This means that</span> |
| <span class="c1"># none of the records within the file belongs to the current source.</span> |
| <span class="k">return</span> |
| |
| <span class="n">_</span><span class="p">,</span> <span class="n">sep_end</span> <span class="o">=</span> <span class="n">sep_bounds</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span> <span class="o">=</span> <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="n">sep_end</span><span class="p">:]</span> |
| <span class="n">next_record_start_position</span> <span class="o">=</span> <span class="n">required_position</span> <span class="o">+</span> <span class="n">sep_end</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">next_record_start_position</span> <span class="o">=</span> <span class="n">position_after_processing_header_lines</span> |
| |
| <span class="k">while</span> <span class="n">range_tracker</span><span class="o">.</span><span class="n">try_claim</span><span class="p">(</span><span class="n">next_record_start_position</span><span class="p">):</span> |
| <span class="n">record</span><span class="p">,</span> <span class="n">num_bytes_to_next_record</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_record</span><span class="p">(</span><span class="n">file_to_read</span><span class="p">,</span> |
| <span class="n">read_buffer</span><span class="p">)</span> |
| <span class="c1"># For compressed text files that use an unsplittable OffsetRangeTracker</span> |
| <span class="c1"># with infinity as the end position, above 'try_claim()' invocation</span> |
| <span class="c1"># would pass for an empty record at the end of file that is not</span> |
| <span class="c1"># followed by a new line character. Since such a record is at the last</span> |
| <span class="c1"># position of a file, it should not be a part of the considered range.</span> |
| <span class="c1"># We do this check to ignore such records.</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">record</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span> <span class="ow">and</span> <span class="n">num_bytes_to_next_record</span> <span class="o"><</span> <span class="mi">0</span><span class="p">:</span> <span class="c1"># pylint: disable=len-as-condition</span> |
| <span class="k">break</span> |
| |
| <span class="c1"># Record delimiter must be larger than zero bytes.</span> |
| <span class="k">assert</span> <span class="n">num_bytes_to_next_record</span> <span class="o">!=</span> <span class="mi">0</span> |
| <span class="k">if</span> <span class="n">num_bytes_to_next_record</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">next_record_start_position</span> <span class="o">+=</span> <span class="n">num_bytes_to_next_record</span> |
| |
| <span class="k">yield</span> <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">decode</span><span class="p">(</span><span class="n">record</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">num_bytes_to_next_record</span> <span class="o"><</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">break</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">_process_header</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">):</span> |
| <span class="c1"># Returns a tuple containing the position in file after processing header</span> |
| <span class="c1"># records and a list of decoded header lines that match</span> |
| <span class="c1"># 'header_matcher'.</span> |
| <span class="n">header_lines</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">position</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_skip_lines</span><span class="p">(</span> |
| <span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_skip_header_lines</span><span class="p">)</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_skip_header_lines</span> <span class="k">else</span> <span class="mi">0</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_header_matcher</span><span class="p">:</span> |
| <span class="k">while</span> <span class="kc">True</span><span class="p">:</span> |
| <span class="n">record</span><span class="p">,</span> <span class="n">num_bytes_to_next_record</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_record</span><span class="p">(</span><span class="n">file_to_read</span><span class="p">,</span> |
| <span class="n">read_buffer</span><span class="p">)</span> |
| <span class="n">decoded_line</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">decode</span><span class="p">(</span><span class="n">record</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_header_matcher</span><span class="p">(</span><span class="n">decoded_line</span><span class="p">):</span> |
| <span class="c1"># We've read past the header section at this point, so go back a line.</span> |
| <span class="n">file_to_read</span><span class="o">.</span><span class="n">seek</span><span class="p">(</span><span class="n">position</span><span class="p">)</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">reset</span><span class="p">()</span> |
| <span class="k">break</span> |
| <span class="n">header_lines</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">decoded_line</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">num_bytes_to_next_record</span> <span class="o"><</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">break</span> |
| <span class="n">position</span> <span class="o">+=</span> <span class="n">num_bytes_to_next_record</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_header_processor</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_header_processor</span><span class="p">(</span><span class="n">header_lines</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">position</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">_find_separator_bounds</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">):</span> |
| <span class="c1"># Determines the start and end positions within 'read_buffer.data' of the</span> |
| <span class="c1"># next delimiter starting from position 'read_buffer.position'.</span> |
| <span class="c1"># Use the custom delimiter to be used in place of</span> |
| <span class="c1"># the default ones ('\n' or '\r\n')'</span> |
| <span class="c1"># This method may increase the size of buffer but it will not decrease the</span> |
| <span class="c1"># size of it.</span> |
| |
| <span class="n">current_pos</span> <span class="o">=</span> <span class="n">read_buffer</span><span class="o">.</span><span class="n">position</span> |
| |
| <span class="c1"># b'\n' use as default</span> |
| <span class="n">delimiter</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_delimiter</span> <span class="ow">or</span> <span class="sa">b</span><span class="s1">'</span><span class="se">\n</span><span class="s1">'</span> |
| <span class="n">delimiter_len</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">delimiter</span><span class="p">)</span> |
| |
| <span class="k">while</span> <span class="kc">True</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">current_pos</span> <span class="o">>=</span> <span class="nb">len</span><span class="p">(</span><span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">)</span> <span class="o">-</span> <span class="n">delimiter_len</span> <span class="o">+</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="c1"># Ensuring that there are enough bytes to determine</span> |
| <span class="c1"># at current_pos.</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_try_to_ensure_num_bytes_in_buffer</span><span class="p">(</span> |
| <span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">,</span> <span class="n">current_pos</span> <span class="o">+</span> <span class="n">delimiter_len</span><span class="p">):</span> |
| <span class="k">return</span> |
| |
| <span class="c1"># Using find() here is more efficient than a linear scan</span> |
| <span class="c1"># of the byte array.</span> |
| <span class="n">next_delim</span> <span class="o">=</span> <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="o">.</span><span class="n">find</span><span class="p">(</span><span class="n">delimiter</span><span class="p">,</span> <span class="n">current_pos</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">next_delim</span> <span class="o">>=</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">if</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_delimiter</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="n">next_delim</span> <span class="o">-</span> <span class="mi">1</span><span class="p">:</span><span class="n">next_delim</span><span class="p">]</span> <span class="o">==</span> <span class="sa">b</span><span class="s1">'</span><span class="se">\r</span><span class="s1">'</span><span class="p">):</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_escapechar</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_escaped</span><span class="p">(</span><span class="n">read_buffer</span><span class="p">,</span> |
| <span class="n">next_delim</span> <span class="o">-</span> <span class="mi">1</span><span class="p">):</span> |
| <span class="c1"># Accept '\n' as a default delimiter, because '\r' is escaped.</span> |
| <span class="k">return</span> <span class="p">(</span><span class="n">next_delim</span><span class="p">,</span> <span class="n">next_delim</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Accept both '\r\n' and '\n' as a default delimiter.</span> |
| <span class="k">return</span> <span class="p">(</span><span class="n">next_delim</span> <span class="o">-</span> <span class="mi">1</span><span class="p">,</span> <span class="n">next_delim</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_escapechar</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_escaped</span><span class="p">(</span><span class="n">read_buffer</span><span class="p">,</span> |
| <span class="n">next_delim</span><span class="p">):</span> |
| <span class="c1"># Skip an escaped delimiter.</span> |
| <span class="n">current_pos</span> <span class="o">=</span> <span class="n">next_delim</span> <span class="o">+</span> <span class="n">delimiter_len</span> <span class="o">+</span> <span class="mi">1</span> |
| <span class="k">continue</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Found a delimiter. Accepting that as the next delimiter.</span> |
| <span class="k">return</span> <span class="p">(</span><span class="n">next_delim</span><span class="p">,</span> <span class="n">next_delim</span> <span class="o">+</span> <span class="n">delimiter_len</span><span class="p">)</span> |
| |
| <span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">_delimiter</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># Corner case: custom delimiter is truncated at the end of the buffer.</span> |
| <span class="n">next_delim</span> <span class="o">=</span> <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="o">.</span><span class="n">find</span><span class="p">(</span> |
| <span class="n">delimiter</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="nb">len</span><span class="p">(</span><span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">)</span> <span class="o">-</span> <span class="n">delimiter_len</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">next_delim</span> <span class="o">>=</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="c1"># Delimiters longer than 1 byte may cross the buffer boundary.</span> |
| <span class="c1"># Defer full matching till the next iteration.</span> |
| <span class="n">current_pos</span> <span class="o">=</span> <span class="n">next_delim</span> |
| <span class="k">continue</span> |
| |
| <span class="n">current_pos</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">)</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">_try_to_ensure_num_bytes_in_buffer</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">,</span> <span class="n">num_bytes</span><span class="p">):</span> |
| <span class="c1"># Tries to ensure that there are at least num_bytes bytes in the buffer.</span> |
| <span class="c1"># Returns True if this can be fulfilled, returned False if this cannot be</span> |
| <span class="c1"># fulfilled due to reaching EOF.</span> |
| <span class="k">while</span> <span class="nb">len</span><span class="p">(</span><span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">)</span> <span class="o"><</span> <span class="n">num_bytes</span><span class="p">:</span> |
| <span class="n">read_data</span> <span class="o">=</span> <span class="n">file_to_read</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">_buffer_size</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">read_data</span><span class="p">:</span> |
| <span class="k">return</span> <span class="kc">False</span> |
| |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span> <span class="o">+=</span> <span class="n">read_data</span> |
| |
| <span class="k">return</span> <span class="kc">True</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">_skip_lines</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">,</span> <span class="n">num_lines</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Skip num_lines from file_to_read, return num_lines+1 start position."""</span> |
| <span class="k">if</span> <span class="n">file_to_read</span><span class="o">.</span><span class="n">tell</span><span class="p">()</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">file_to_read</span><span class="o">.</span><span class="n">seek</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span> |
| <span class="n">position</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">num_lines</span><span class="p">):</span> |
| <span class="n">_</span><span class="p">,</span> <span class="n">num_bytes_to_next_record</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_record</span><span class="p">(</span><span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">num_bytes_to_next_record</span> <span class="o"><</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="c1"># We reached end of file. It is OK to just break here</span> |
| <span class="c1"># because subsequent _read_record will return same result.</span> |
| <span class="k">break</span> |
| <span class="n">position</span> <span class="o">+=</span> <span class="n">num_bytes_to_next_record</span> |
| <span class="k">return</span> <span class="n">position</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">_read_record</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">):</span> |
| <span class="c1"># Returns a tuple containing the current_record and number of bytes to the</span> |
| <span class="c1"># next record starting from 'read_buffer.position'. If EOF is</span> |
| <span class="c1"># reached, returns a tuple containing the current record and -1.</span> |
| |
| <span class="k">if</span> <span class="n">read_buffer</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">_buffer_size</span><span class="p">:</span> |
| <span class="c1"># read_buffer is too large. Truncating and adjusting it.</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span> <span class="o">=</span> <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="n">read_buffer</span><span class="o">.</span><span class="n">position</span><span class="p">:]</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">position</span> <span class="o">=</span> <span class="mi">0</span> |
| |
| <span class="n">record_start_position_in_buffer</span> <span class="o">=</span> <span class="n">read_buffer</span><span class="o">.</span><span class="n">position</span> |
| <span class="n">sep_bounds</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_find_separator_bounds</span><span class="p">(</span><span class="n">file_to_read</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">)</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">position</span> <span class="o">=</span> <span class="n">sep_bounds</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span> <span class="k">if</span> <span class="n">sep_bounds</span> <span class="k">else</span> <span class="nb">len</span><span class="p">(</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">sep_bounds</span><span class="p">:</span> |
| <span class="c1"># Reached EOF. Bytes up to the EOF is the next record. Returning '-1' for</span> |
| <span class="c1"># the starting position of the next record.</span> |
| <span class="k">return</span> <span class="p">(</span><span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="n">record_start_position_in_buffer</span><span class="p">:],</span> <span class="o">-</span><span class="mi">1</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_strip_trailing_newlines</span><span class="p">:</span> |
| <span class="c1"># Current record should not contain the delimiter.</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="n">record_start_position_in_buffer</span><span class="p">:</span><span class="n">sep_bounds</span><span class="p">[</span><span class="mi">0</span><span class="p">]],</span> |
| <span class="n">sep_bounds</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span> <span class="o">-</span> <span class="n">record_start_position_in_buffer</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Current record should contain the delimiter.</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="n">record_start_position_in_buffer</span><span class="p">:</span><span class="n">sep_bounds</span><span class="p">[</span><span class="mi">1</span><span class="p">]],</span> |
| <span class="n">sep_bounds</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span> <span class="o">-</span> <span class="n">record_start_position_in_buffer</span><span class="p">)</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">_is_self_overlapping</span><span class="p">(</span><span class="n">delimiter</span><span class="p">):</span> |
| <span class="c1"># A delimiter self-overlaps if it has a prefix that is also its suffix.</span> |
| <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="nb">len</span><span class="p">(</span><span class="n">delimiter</span><span class="p">)):</span> |
| <span class="k">if</span> <span class="n">delimiter</span><span class="p">[</span><span class="mi">0</span><span class="p">:</span><span class="n">i</span><span class="p">]</span> <span class="o">==</span> <span class="n">delimiter</span><span class="p">[</span><span class="nb">len</span><span class="p">(</span><span class="n">delimiter</span><span class="p">)</span> <span class="o">-</span> <span class="n">i</span><span class="p">:]:</span> |
| <span class="k">return</span> <span class="kc">True</span> |
| <span class="k">return</span> <span class="kc">False</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">_is_escaped</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">read_buffer</span><span class="p">,</span> <span class="n">position</span><span class="p">):</span> |
| <span class="c1"># Returns True if byte at position is preceded with an odd number</span> |
| <span class="c1"># of escapechar bytes or False if preceded by 0 or even escapes</span> |
| <span class="c1"># (the even number means that all the escapes are escaped themselves).</span> |
| <span class="n">escape_count</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="k">for</span> <span class="n">current_pos</span> <span class="ow">in</span> <span class="nb">reversed</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">position</span><span class="p">)):</span> |
| <span class="k">if</span> <span class="n">read_buffer</span><span class="o">.</span><span class="n">data</span><span class="p">[</span><span class="n">current_pos</span><span class="p">:</span><span class="n">current_pos</span> <span class="o">+</span> <span class="mi">1</span><span class="p">]</span> <span class="o">!=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_escapechar</span><span class="p">:</span> |
| <span class="k">break</span> |
| <span class="n">escape_count</span> <span class="o">+=</span> <span class="mi">1</span> |
| <span class="k">return</span> <span class="n">escape_count</span> <span class="o">%</span> <span class="mi">2</span> <span class="o">==</span> <span class="mi">1</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">output_type_hint</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_coder</span><span class="o">.</span><span class="n">to_type_hint</span><span class="p">()</span> |
| <span class="k">except</span> <span class="ne">NotImplementedError</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">Any</span> |
| |
| |
| <span class="k">class</span><span class="w"> </span><span class="nc">_TextSourceWithFilename</span><span class="p">(</span><span class="n">_TextSource</span><span class="p">):</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">read_records</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_name</span><span class="p">,</span> <span class="n">range_tracker</span><span class="p">):</span> |
| <span class="n">records</span> <span class="o">=</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">read_records</span><span class="p">(</span><span class="n">file_name</span><span class="p">,</span> <span class="n">range_tracker</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">record</span> <span class="ow">in</span> <span class="n">records</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="p">(</span><span class="n">file_name</span><span class="p">,</span> <span class="n">record</span><span class="p">)</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">output_type_hint</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">typehints</span><span class="o">.</span><span class="n">KV</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">output_type_hint</span><span class="p">()]</span> |
| |
| |
| <span class="k">class</span><span class="w"> </span><span class="nc">_TextSink</span><span class="p">(</span><span class="n">filebasedsink</span><span class="o">.</span><span class="n">FileBasedSink</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A sink to a GCS or local text file or files."""</span> |
| <span class="k">def</span><span class="w"> </span><span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">file_path_prefix</span><span class="p">,</span> |
| <span class="n">file_name_suffix</span><span class="o">=</span><span class="s1">''</span><span class="p">,</span> |
| <span class="n">append_trailing_newlines</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">num_shards</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">shard_name_template</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">:</span> <span class="n">coders</span><span class="o">.</span><span class="n">Coder</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">ToBytesCoder</span><span class="p">(),</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">CompressionTypes</span><span class="o">.</span><span class="n">AUTO</span><span class="p">,</span> |
| <span class="n">header</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">footer</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="o">*</span><span class="p">,</span> |
| <span class="n">max_records_per_shard</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">max_bytes_per_shard</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">skip_if_empty</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Initialize a _TextSink.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> file_path_prefix: The file path to write to. The files written will begin</span> |
| <span class="sd"> with this prefix, followed by a shard identifier (see num_shards), and</span> |
| <span class="sd"> end in a common extension, if given by file_name_suffix. In most cases,</span> |
| <span class="sd"> only this argument is specified and num_shards, shard_name_template, and</span> |
| <span class="sd"> file_name_suffix use default values.</span> |
| <span class="sd"> file_name_suffix: Suffix for the files written.</span> |
| <span class="sd"> append_trailing_newlines: indicate whether this sink should write an</span> |
| <span class="sd"> additional newline char after writing each element.</span> |
| <span class="sd"> num_shards: The number of files (shards) used for output. If not set, the</span> |
| <span class="sd"> service will decide on the optimal number of shards.</span> |
| <span class="sd"> Constraining the number of shards is likely to reduce</span> |
| <span class="sd"> the performance of a pipeline. Setting this value is not recommended</span> |
| <span class="sd"> unless you require a specific number of output files.</span> |
| <span class="sd"> shard_name_template: A template string containing placeholders for</span> |
| <span class="sd"> the shard number and shard count. When constructing a filename for a</span> |
| <span class="sd"> particular shard number, the upper-case letters 'S' and 'N' are</span> |
| <span class="sd"> replaced with the 0-padded shard number and shard count respectively.</span> |
| <span class="sd"> This argument can be '' in which case it behaves as if num_shards was</span> |
| <span class="sd"> set to 1 and only one file will be generated. The default pattern used</span> |
| <span class="sd"> is '-SSSSS-of-NNNNN' if None is passed as the shard_name_template.</span> |
| <span class="sd"> coder: Coder used to encode each line.</span> |
| <span class="sd"> compression_type: Used to handle compressed output files. Typical value</span> |
| <span class="sd"> is CompressionTypes.AUTO, in which case the final file path's</span> |
| <span class="sd"> extension (as determined by file_path_prefix, file_name_suffix,</span> |
| <span class="sd"> num_shards and shard_name_template) will be used to detect the</span> |
| <span class="sd"> compression.</span> |
| <span class="sd"> header: String to write at beginning of file as a header. If not None and</span> |
| <span class="sd"> append_trailing_newlines is set, '\n' will be added.</span> |
| <span class="sd"> footer: String to write at the end of file as a footer. If not None and</span> |
| <span class="sd"> append_trailing_newlines is set, '\n' will be added.</span> |
| <span class="sd"> max_records_per_shard: Maximum number of records to write to any</span> |
| <span class="sd"> individual shard.</span> |
| <span class="sd"> max_bytes_per_shard: Target maximum number of bytes to write to any</span> |
| <span class="sd"> individual shard. This may be exceeded slightly, as a new shard is</span> |
| <span class="sd"> created once this limit is hit, but the remainder of a given record, a</span> |
| <span class="sd"> subsequent newline, and a footer may cause the actual shard size</span> |
| <span class="sd"> to exceed this value. This also tracks the uncompressed,</span> |
| <span class="sd"> not compressed, size of the shard.</span> |
| <span class="sd"> skip_if_empty: Don't write any shards if the PCollection is empty.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> A _TextSink object usable for writing.</span> |
| <span class="sd"> """</span> |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span> |
| <span class="n">file_path_prefix</span><span class="p">,</span> |
| <span class="n">file_name_suffix</span><span class="o">=</span><span class="n">file_name_suffix</span><span class="p">,</span> |
| <span class="n">num_shards</span><span class="o">=</span><span class="n">num_shards</span><span class="p">,</span> |
| <span class="n">shard_name_template</span><span class="o">=</span><span class="n">shard_name_template</span><span class="p">,</span> |
| <span class="n">coder</span><span class="o">=</span><span class="n">coder</span><span class="p">,</span> |
| <span class="n">mime_type</span><span class="o">=</span><span class="s1">'text/plain'</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">compression_type</span><span class="p">,</span> |
| <span class="n">max_records_per_shard</span><span class="o">=</span><span class="n">max_records_per_shard</span><span class="p">,</span> |
| <span class="n">max_bytes_per_shard</span><span class="o">=</span><span class="n">max_bytes_per_shard</span><span class="p">,</span> |
| <span class="n">skip_if_empty</span><span class="o">=</span><span class="n">skip_if_empty</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_append_trailing_newlines</span> <span class="o">=</span> <span class="n">append_trailing_newlines</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_header</span> <span class="o">=</span> <span class="n">header</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_footer</span> <span class="o">=</span> <span class="n">footer</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">open</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">temp_path</span><span class="p">):</span> |
| <span class="n">file_handle</span> <span class="o">=</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">open</span><span class="p">(</span><span class="n">temp_path</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_header</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">file_handle</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="n">coders</span><span class="o">.</span><span class="n">ToBytesCoder</span><span class="p">()</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_header</span><span class="p">))</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_append_trailing_newlines</span><span class="p">:</span> |
| <span class="n">file_handle</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="sa">b</span><span class="s1">'</span><span class="se">\n</span><span class="s1">'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">file_handle</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">close</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_handle</span><span class="p">):</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_footer</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">file_handle</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="n">coders</span><span class="o">.</span><span class="n">ToBytesCoder</span><span class="p">()</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_footer</span><span class="p">))</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_append_trailing_newlines</span><span class="p">:</span> |
| <span class="n">file_handle</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="sa">b</span><span class="s1">'</span><span class="se">\n</span><span class="s1">'</span><span class="p">)</span> |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">close</span><span class="p">(</span><span class="n">file_handle</span><span class="p">)</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="n">dd_parent</span> <span class="o">=</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">display_data</span><span class="p">()</span> |
| <span class="n">dd_parent</span><span class="p">[</span><span class="s1">'append_newline'</span><span class="p">]</span> <span class="o">=</span> <span class="n">DisplayDataItem</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_append_trailing_newlines</span><span class="p">,</span> <span class="n">label</span><span class="o">=</span><span class="s1">'Append Trailing New Lines'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">dd_parent</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">write_encoded_record</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_handle</span><span class="p">,</span> <span class="n">encoded_value</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Writes a single encoded record."""</span> |
| <span class="n">file_handle</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="n">encoded_value</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_append_trailing_newlines</span><span class="p">:</span> |
| <span class="n">file_handle</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="sa">b</span><span class="s1">'</span><span class="se">\n</span><span class="s1">'</span><span class="p">)</span> |
| |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">_create_text_source</span><span class="p">(</span> |
| <span class="n">file_pattern</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">strip_trailing_newlines</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">coder</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">validate</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">skip_header_lines</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">delimiter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">escapechar</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_TextSource</span><span class="p">(</span> |
| <span class="n">file_pattern</span><span class="o">=</span><span class="n">file_pattern</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="o">=</span><span class="n">min_bundle_size</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">compression_type</span><span class="p">,</span> |
| <span class="n">strip_trailing_newlines</span><span class="o">=</span><span class="n">strip_trailing_newlines</span><span class="p">,</span> |
| <span class="n">coder</span><span class="o">=</span><span class="n">coder</span><span class="p">,</span> |
| <span class="n">validate</span><span class="o">=</span><span class="n">validate</span><span class="p">,</span> |
| <span class="n">skip_header_lines</span><span class="o">=</span><span class="n">skip_header_lines</span><span class="p">,</span> |
| <span class="n">delimiter</span><span class="o">=</span><span class="n">delimiter</span><span class="p">,</span> |
| <span class="n">escapechar</span><span class="o">=</span><span class="n">escapechar</span><span class="p">)</span> |
| |
| |
| <div class="viewcode-block" id="ReadAllFromText"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadAllFromText">[docs]</a> |
| <span class="k">class</span><span class="w"> </span><span class="nc">ReadAllFromText</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A ``PTransform`` for reading a ``PCollection`` of text files.</span> |
| |
| <span class="sd"> Reads a ``PCollection`` of text files or file patterns and produces a</span> |
| <span class="sd"> ``PCollection`` of strings.</span> |
| |
| <span class="sd"> Parses a text file as newline-delimited elements, by default assuming</span> |
| <span class="sd"> UTF-8 encoding. Supports newline delimiters '\\n' and '\\r\\n'.</span> |
| |
| <span class="sd"> If `with_filename` is ``True`` the output will include the file name. This is</span> |
| <span class="sd"> similar to ``ReadFromTextWithFilename`` but this ``PTransform`` can be placed</span> |
| <span class="sd"> anywhere in the pipeline.</span> |
| |
| <span class="sd"> If reading from a text file that that requires a different encoding, you may</span> |
| <span class="sd"> provide a custom :class:`~apache_beam.coders.coders.Coder` that encodes and</span> |
| <span class="sd"> decodes with the appropriate codec. For example, see the implementation of</span> |
| <span class="sd"> :class:`~apache_beam.coders.coders.StrUtf8Coder`.</span> |
| |
| <span class="sd"> This does not support ``UTF-16`` or ``UTF-32`` encodings.</span> |
| |
| <span class="sd"> This implementation is only tested with batch pipeline. In streaming,</span> |
| <span class="sd"> reading may happen with delay due to the limitation in ReShuffle involved.</span> |
| <span class="sd"> """</span> |
| <span class="n">DEFAULT_DESIRED_BUNDLE_SIZE</span> <span class="o">=</span> <span class="mi">64</span> <span class="o">*</span> <span class="mi">1024</span> <span class="o">*</span> <span class="mi">1024</span> <span class="c1"># 64MB</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">desired_bundle_size</span><span class="o">=</span><span class="n">DEFAULT_DESIRED_BUNDLE_SIZE</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">CompressionTypes</span><span class="o">.</span><span class="n">AUTO</span><span class="p">,</span> |
| <span class="n">strip_trailing_newlines</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">validate</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">:</span> <span class="n">coders</span><span class="o">.</span><span class="n">Coder</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">StrUtf8Coder</span><span class="p">(),</span> |
| <span class="n">skip_header_lines</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">with_filename</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">delimiter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">escapechar</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Initialize the ``ReadAllFromText`` transform.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> min_bundle_size: Minimum size of bundles that should be generated when</span> |
| <span class="sd"> splitting this source into bundles. See ``FileBasedSource`` for more</span> |
| <span class="sd"> details.</span> |
| <span class="sd"> desired_bundle_size: Desired size of bundles that should be generated when</span> |
| <span class="sd"> splitting this source into bundles. See ``FileBasedSource`` for more</span> |
| <span class="sd"> details.</span> |
| <span class="sd"> compression_type: Used to handle compressed input files. Typical value</span> |
| <span class="sd"> is ``CompressionTypes.AUTO``, in which case the underlying file_path's</span> |
| <span class="sd"> extension will be used to detect the compression.</span> |
| <span class="sd"> strip_trailing_newlines: Indicates whether this source should remove</span> |
| <span class="sd"> the newline char in each line it reads before decoding that line.</span> |
| <span class="sd"> validate: flag to verify that the files exist during the pipeline</span> |
| <span class="sd"> creation time.</span> |
| <span class="sd"> skip_header_lines: Number of header lines to skip. Same number is skipped</span> |
| <span class="sd"> from each source file. Must be 0 or higher. Large number of skipped</span> |
| <span class="sd"> lines might impact performance.</span> |
| <span class="sd"> coder: Coder used to decode each line.</span> |
| <span class="sd"> with_filename: If True, returns a Key Value with the key being the file</span> |
| <span class="sd"> name and the value being the actual data. If False, it only returns</span> |
| <span class="sd"> the data.</span> |
| <span class="sd"> delimiter (bytes) Optional: delimiter to split records.</span> |
| <span class="sd"> Must not self-overlap, because self-overlapping delimiters cause</span> |
| <span class="sd"> ambiguous parsing.</span> |
| <span class="sd"> escapechar (bytes) Optional: a single byte to escape the records</span> |
| <span class="sd"> delimiter, can also escape itself.</span> |
| <span class="sd"> """</span> |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_source_from_file</span> <span class="o">=</span> <span class="n">partial</span><span class="p">(</span> |
| <span class="n">_create_text_source</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="o">=</span><span class="n">min_bundle_size</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">compression_type</span><span class="p">,</span> |
| <span class="n">strip_trailing_newlines</span><span class="o">=</span><span class="n">strip_trailing_newlines</span><span class="p">,</span> |
| <span class="n">validate</span><span class="o">=</span><span class="n">validate</span><span class="p">,</span> |
| <span class="n">coder</span><span class="o">=</span><span class="n">coder</span><span class="p">,</span> |
| <span class="n">skip_header_lines</span><span class="o">=</span><span class="n">skip_header_lines</span><span class="p">,</span> |
| <span class="n">delimiter</span><span class="o">=</span><span class="n">delimiter</span><span class="p">,</span> |
| <span class="n">escapechar</span><span class="o">=</span><span class="n">escapechar</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_desired_bundle_size</span> <span class="o">=</span> <span class="n">desired_bundle_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_min_bundle_size</span> <span class="o">=</span> <span class="n">min_bundle_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_compression_type</span> <span class="o">=</span> <span class="n">compression_type</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_with_filename</span> <span class="o">=</span> <span class="n">with_filename</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_read_all_files</span> <span class="o">=</span> <span class="n">ReadAllFiles</span><span class="p">(</span> |
| <span class="kc">True</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_compression_type</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_desired_bundle_size</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_min_bundle_size</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_source_from_file</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_with_filename</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="ReadAllFromText.expand"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadAllFromText.expand">[docs]</a> |
| <span class="k">def</span><span class="w"> </span><span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalue</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pvalue</span> <span class="o">|</span> <span class="s1">'ReadAllFiles'</span> <span class="o">>></span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_all_files</span></div> |
| </div> |
| |
| |
| |
| <div class="viewcode-block" id="ReadAllFromTextContinuously"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadAllFromTextContinuously">[docs]</a> |
| <span class="k">class</span><span class="w"> </span><span class="nc">ReadAllFromTextContinuously</span><span class="p">(</span><span class="n">ReadAllFromText</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A ``PTransform`` for reading text files in given file patterns.</span> |
| <span class="sd"> This PTransform acts as a Source and produces continuously a ``PCollection``</span> |
| <span class="sd"> of strings.</span> |
| |
| <span class="sd"> For more details, see ``ReadAllFromText`` for text parsing settings;</span> |
| <span class="sd"> see ``apache_beam.io.fileio.MatchContinuously`` for watching settings.</span> |
| |
| <span class="sd"> ReadAllFromTextContinuously is experimental. No backwards-compatibility</span> |
| <span class="sd"> guarantees. Due to the limitation on Reshuffle, current implementation does</span> |
| <span class="sd"> not scale.</span> |
| <span class="sd"> """</span> |
| <span class="n">_ARGS_FOR_MATCH</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s1">'interval'</span><span class="p">,</span> |
| <span class="s1">'has_deduplication'</span><span class="p">,</span> |
| <span class="s1">'start_timestamp'</span><span class="p">,</span> |
| <span class="s1">'stop_timestamp'</span><span class="p">,</span> |
| <span class="s1">'match_updated_files'</span><span class="p">,</span> |
| <span class="s1">'apply_windowing'</span><span class="p">)</span> |
| <span class="n">_ARGS_FOR_READ</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s1">'min_bundle_size'</span><span class="p">,</span> |
| <span class="s1">'desired_bundle_size'</span><span class="p">,</span> |
| <span class="s1">'compression_type'</span><span class="p">,</span> |
| <span class="s1">'strip_trailing_newlines'</span><span class="p">,</span> |
| <span class="s1">'validate'</span><span class="p">,</span> |
| <span class="s1">'coder'</span><span class="p">,</span> |
| <span class="s1">'skip_header_lines'</span><span class="p">,</span> |
| <span class="s1">'with_filename'</span><span class="p">,</span> |
| <span class="s1">'delimiter'</span><span class="p">,</span> |
| <span class="s1">'escapechar'</span><span class="p">)</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_pattern</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Initialize the ``ReadAllFromTextContinuously`` transform.</span> |
| |
| <span class="sd"> Accepts args for constructor args of both :class:`ReadAllFromText` and</span> |
| <span class="sd"> :class:`~apache_beam.io.fileio.MatchContinuously`.</span> |
| <span class="sd"> """</span> |
| <span class="n">kwargs_for_match</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">k</span><span class="p">:</span> <span class="n">v</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">)</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> <span class="k">if</span> <span class="n">k</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ARGS_FOR_MATCH</span> |
| <span class="p">}</span> |
| <span class="n">kwargs_for_read</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">k</span><span class="p">:</span> <span class="n">v</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">)</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> <span class="k">if</span> <span class="n">k</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ARGS_FOR_READ</span> |
| <span class="p">}</span> |
| <span class="n">kwargs_additinal</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">k</span><span class="p">:</span> <span class="n">v</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">)</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="k">if</span> <span class="n">k</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ARGS_FOR_MATCH</span> <span class="ow">and</span> <span class="n">k</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ARGS_FOR_READ</span> |
| <span class="p">}</span> |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs_for_read</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs_additinal</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_file_pattern</span> <span class="o">=</span> <span class="n">file_pattern</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_kwargs_for_match</span> <span class="o">=</span> <span class="n">kwargs_for_match</span> |
| |
| <div class="viewcode-block" id="ReadAllFromTextContinuously.expand"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadAllFromTextContinuously.expand">[docs]</a> |
| <span class="k">def</span><span class="w"> </span><span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pbegin</span><span class="p">):</span> |
| <span class="c1"># Importing locally to prevent circular dependency issues.</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.io.fileio</span><span class="w"> </span><span class="kn">import</span> <span class="n">MatchContinuously</span> |
| |
| <span class="c1"># TODO(BEAM-14497) always reshuffle once gbk always trigger works.</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">pbegin</span> |
| <span class="o">|</span> <span class="n">MatchContinuously</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_file_pattern</span><span class="p">,</span> <span class="o">**</span><span class="bp">self</span><span class="o">.</span><span class="n">_kwargs_for_match</span><span class="p">)</span> |
| <span class="o">|</span> <span class="s1">'ReadAllFiles'</span> <span class="o">>></span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_all_files</span><span class="o">.</span><span class="n">_disable_reshuffle</span><span class="p">())</span></div> |
| </div> |
| |
| |
| |
| <div class="viewcode-block" id="ReadFromText"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadFromText">[docs]</a> |
| <span class="k">class</span><span class="w"> </span><span class="nc">ReadFromText</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sa">r</span><span class="sd">"""A :class:`~apache_beam.transforms.ptransform.PTransform` for reading text</span> |
| <span class="sd"> files.</span> |
| |
| <span class="sd"> Parses a text file as newline-delimited elements, by default assuming</span> |
| <span class="sd"> ``UTF-8`` encoding. Supports newline delimiters ``\n`` and ``\r\n``</span> |
| <span class="sd"> or specified delimiter.</span> |
| |
| <span class="sd"> If reading from a text file that that requires a different encoding, you may</span> |
| <span class="sd"> provide a custom :class:`~apache_beam.coders.coders.Coder` that encodes and</span> |
| <span class="sd"> decodes with the appropriate codec. For example, see the implementation of</span> |
| <span class="sd"> :class:`~apache_beam.coders.coders.StrUtf8Coder`.</span> |
| |
| <span class="sd"> This does not support ``UTF-16`` or ``UTF-32`` encodings.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">_source_class</span> <span class="o">=</span> <span class="n">_TextSource</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">file_pattern</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">CompressionTypes</span><span class="o">.</span><span class="n">AUTO</span><span class="p">,</span> |
| <span class="n">strip_trailing_newlines</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">:</span> <span class="n">coders</span><span class="o">.</span><span class="n">Coder</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">StrUtf8Coder</span><span class="p">(),</span> |
| <span class="n">validate</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">skip_header_lines</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">delimiter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">escapechar</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Initialize the :class:`ReadFromText` transform.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> file_pattern (str): The file path to read from as a local file path or a</span> |
| <span class="sd"> GCS ``gs://`` path. The path can contain glob characters</span> |
| <span class="sd"> (``*``, ``?``, and ``[...]`` sets).</span> |
| <span class="sd"> min_bundle_size (int): Minimum size of bundles that should be generated</span> |
| <span class="sd"> when splitting this source into bundles. See</span> |
| <span class="sd"> :class:`~apache_beam.io.filebasedsource.FileBasedSource` for more</span> |
| <span class="sd"> details.</span> |
| <span class="sd"> compression_type (str): Used to handle compressed input files.</span> |
| <span class="sd"> Typical value is :attr:`CompressionTypes.AUTO</span> |
| <span class="sd"> <apache_beam.io.filesystem.CompressionTypes.AUTO>`, in which case the</span> |
| <span class="sd"> underlying file_path's extension will be used to detect the compression.</span> |
| <span class="sd"> strip_trailing_newlines (bool): Indicates whether this source should</span> |
| <span class="sd"> remove the newline char in each line it reads before decoding that line.</span> |
| <span class="sd"> validate (bool): flag to verify that the files exist during the pipeline</span> |
| <span class="sd"> creation time.</span> |
| <span class="sd"> skip_header_lines (int): Number of header lines to skip. Same number is</span> |
| <span class="sd"> skipped from each source file. Must be 0 or higher. Large number of</span> |
| <span class="sd"> skipped lines might impact performance.</span> |
| <span class="sd"> coder (~apache_beam.coders.coders.Coder): Coder used to decode each line.</span> |
| <span class="sd"> delimiter (bytes) Optional: delimiter to split records.</span> |
| <span class="sd"> Must not self-overlap, because self-overlapping delimiters cause</span> |
| <span class="sd"> ambiguous parsing.</span> |
| <span class="sd"> escapechar (bytes) Optional: a single byte to escape the records</span> |
| <span class="sd"> delimiter, can also escape itself.</span> |
| <span class="sd"> """</span> |
| |
| <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">file_pattern</span><span class="p">:</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">dirname</span><span class="p">(</span><span class="n">file_pattern</span><span class="p">):</span> |
| <span class="n">file_pattern</span> <span class="o">=</span> <span class="n">os</span><span class="o">.</span><span class="n">path</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="s1">'.'</span><span class="p">,</span> <span class="n">file_pattern</span><span class="p">)</span> |
| <span class="k">except</span> <span class="ne">TypeError</span><span class="p">:</span> |
| <span class="k">pass</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_source</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_source_class</span><span class="p">(</span> |
| <span class="n">file_pattern</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="p">,</span> |
| <span class="n">strip_trailing_newlines</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">,</span> |
| <span class="n">validate</span><span class="o">=</span><span class="n">validate</span><span class="p">,</span> |
| <span class="n">skip_header_lines</span><span class="o">=</span><span class="n">skip_header_lines</span><span class="p">,</span> |
| <span class="n">delimiter</span><span class="o">=</span><span class="n">delimiter</span><span class="p">,</span> |
| <span class="n">escapechar</span><span class="o">=</span><span class="n">escapechar</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="ReadFromText.expand"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadFromText.expand">[docs]</a> |
| <span class="k">def</span><span class="w"> </span><span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalue</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pvalue</span><span class="o">.</span><span class="n">pipeline</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">_source</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_source</span><span class="o">.</span><span class="n">output_type_hint</span><span class="p">())</span></div> |
| </div> |
| |
| |
| |
| <div class="viewcode-block" id="ReadFromTextWithFilename"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadFromTextWithFilename">[docs]</a> |
| <span class="k">class</span><span class="w"> </span><span class="nc">ReadFromTextWithFilename</span><span class="p">(</span><span class="n">ReadFromText</span><span class="p">):</span> |
| <span class="w"> </span><span class="sa">r</span><span class="sd">"""A :class:`~apache_beam.io.textio.ReadFromText` for reading text</span> |
| <span class="sd"> files returning the name of the file and the content of the file.</span> |
| |
| <span class="sd"> This class extend ReadFromText class just setting a different</span> |
| <span class="sd"> _source_class attribute.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">_source_class</span> <span class="o">=</span> <span class="n">_TextSourceWithFilename</span></div> |
| |
| |
| |
| <div class="viewcode-block" id="WriteToText"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.WriteToText">[docs]</a> |
| <span class="k">class</span><span class="w"> </span><span class="nc">WriteToText</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A :class:`~apache_beam.transforms.ptransform.PTransform` for writing to</span> |
| <span class="sd"> text files."""</span> |
| <span class="k">def</span><span class="w"> </span><span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">file_path_prefix</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> |
| <span class="n">file_name_suffix</span><span class="o">=</span><span class="s1">''</span><span class="p">,</span> |
| <span class="n">append_trailing_newlines</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">num_shards</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> |
| <span class="n">shard_name_template</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">:</span> <span class="n">coders</span><span class="o">.</span><span class="n">Coder</span> <span class="o">=</span> <span class="n">coders</span><span class="o">.</span><span class="n">ToBytesCoder</span><span class="p">(),</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">CompressionTypes</span><span class="o">.</span><span class="n">AUTO</span><span class="p">,</span> |
| <span class="n">header</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">footer</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="o">*</span><span class="p">,</span> |
| <span class="n">max_records_per_shard</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">max_bytes_per_shard</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">skip_if_empty</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="w"> </span><span class="sa">r</span><span class="sd">"""Initialize a :class:`WriteToText` transform.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> file_path_prefix (str): The file path to write to. The files written will</span> |
| <span class="sd"> begin with this prefix, followed by a shard identifier (see</span> |
| <span class="sd"> **num_shards**), and end in a common extension, if given by</span> |
| <span class="sd"> **file_name_suffix**. In most cases, only this argument is specified and</span> |
| <span class="sd"> **num_shards**, **shard_name_template**, and **file_name_suffix** use</span> |
| <span class="sd"> default values.</span> |
| <span class="sd"> file_name_suffix (str): Suffix for the files written.</span> |
| <span class="sd"> append_trailing_newlines (bool): indicate whether this sink should write</span> |
| <span class="sd"> an additional newline char after writing each element.</span> |
| <span class="sd"> num_shards (int): The number of files (shards) used for output.</span> |
| <span class="sd"> If not set, the service will decide on the optimal number of shards.</span> |
| <span class="sd"> Constraining the number of shards is likely to reduce</span> |
| <span class="sd"> the performance of a pipeline. Setting this value is not recommended</span> |
| <span class="sd"> unless you require a specific number of output files.</span> |
| <span class="sd"> shard_name_template (str): A template string containing placeholders for</span> |
| <span class="sd"> the shard number and shard count. Currently only ``''`` and</span> |
| <span class="sd"> ``'-SSSSS-of-NNNNN'`` are patterns accepted by the service.</span> |
| <span class="sd"> When constructing a filename for a particular shard number, the</span> |
| <span class="sd"> upper-case letters ``S`` and ``N`` are replaced with the ``0``-padded</span> |
| <span class="sd"> shard number and shard count respectively. This argument can be ``''``</span> |
| <span class="sd"> in which case it behaves as if num_shards was set to 1 and only one file</span> |
| <span class="sd"> will be generated. The default pattern used is ``'-SSSSS-of-NNNNN'``.</span> |
| <span class="sd"> coder (~apache_beam.coders.coders.Coder): Coder used to encode each line.</span> |
| <span class="sd"> compression_type (str): Used to handle compressed output files.</span> |
| <span class="sd"> Typical value is :class:`CompressionTypes.AUTO</span> |
| <span class="sd"> <apache_beam.io.filesystem.CompressionTypes.AUTO>`, in which case the</span> |
| <span class="sd"> final file path's extension (as determined by **file_path_prefix**,</span> |
| <span class="sd"> **file_name_suffix**, **num_shards** and **shard_name_template**) will</span> |
| <span class="sd"> be used to detect the compression.</span> |
| <span class="sd"> header (str): String to write at beginning of file as a header.</span> |
| <span class="sd"> If not :data:`None` and **append_trailing_newlines** is set, ``\n`` will</span> |
| <span class="sd"> be added.</span> |
| <span class="sd"> footer (str): String to write at the end of file as a footer.</span> |
| <span class="sd"> If not :data:`None` and **append_trailing_newlines** is set, ``\n`` will</span> |
| <span class="sd"> be added.</span> |
| <span class="sd"> max_records_per_shard: Maximum number of records to write to any</span> |
| <span class="sd"> individual shard.</span> |
| <span class="sd"> max_bytes_per_shard: Target maximum number of bytes to write to any</span> |
| <span class="sd"> individual shard. This may be exceeded slightly, as a new shard is</span> |
| <span class="sd"> created once this limit is hit, but the remainder of a given record, a</span> |
| <span class="sd"> subsequent newline, and a footer may cause the actual shard size</span> |
| <span class="sd"> to exceed this value. This also tracks the uncompressed,</span> |
| <span class="sd"> not compressed, size of the shard.</span> |
| <span class="sd"> skip_if_empty: Don't write any shards if the PCollection is empty.</span> |
| <span class="sd"> In case of an empty PCollection, this will still delete existing</span> |
| <span class="sd"> files having same file path and not create new ones.</span> |
| <span class="sd"> """</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_sink</span> <span class="o">=</span> <span class="n">_TextSink</span><span class="p">(</span> |
| <span class="n">file_path_prefix</span><span class="p">,</span> |
| <span class="n">file_name_suffix</span><span class="p">,</span> |
| <span class="n">append_trailing_newlines</span><span class="p">,</span> |
| <span class="n">num_shards</span><span class="p">,</span> |
| <span class="n">shard_name_template</span><span class="p">,</span> |
| <span class="n">coder</span><span class="p">,</span> |
| <span class="n">compression_type</span><span class="p">,</span> |
| <span class="n">header</span><span class="p">,</span> |
| <span class="n">footer</span><span class="p">,</span> |
| <span class="n">max_records_per_shard</span><span class="o">=</span><span class="n">max_records_per_shard</span><span class="p">,</span> |
| <span class="n">max_bytes_per_shard</span><span class="o">=</span><span class="n">max_bytes_per_shard</span><span class="p">,</span> |
| <span class="n">skip_if_empty</span><span class="o">=</span><span class="n">skip_if_empty</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="WriteToText.expand"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.WriteToText.expand">[docs]</a> |
| <span class="k">def</span><span class="w"> </span><span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="n">Write</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_sink</span><span class="p">)</span></div> |
| </div> |
| |
| |
| |
| <span class="k">try</span><span class="p">:</span> |
| <span class="kn">import</span><span class="w"> </span><span class="nn">pandas</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">append_pandas_args</span><span class="p">(</span><span class="n">src</span><span class="p">,</span> <span class="n">exclude</span><span class="p">):</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">append</span><span class="p">(</span><span class="n">dest</span><span class="p">):</span> |
| <span class="n">state</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="n">skip</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="n">extra_lines</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">line</span> <span class="ow">in</span> <span class="n">src</span><span class="o">.</span><span class="vm">__doc__</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">'</span><span class="se">\n</span><span class="s1">'</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">line</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span> <span class="o">==</span> <span class="s1">'Parameters'</span><span class="p">:</span> |
| <span class="n">indent</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">line</span><span class="p">)</span> <span class="o">-</span> <span class="nb">len</span><span class="p">(</span><span class="n">line</span><span class="o">.</span><span class="n">lstrip</span><span class="p">())</span> |
| <span class="n">extra_lines</span> <span class="o">=</span> <span class="p">[</span><span class="s1">'</span><span class="se">\n\n</span><span class="s1">Pandas Parameters'</span><span class="p">]</span> |
| <span class="n">state</span> <span class="o">=</span> <span class="s1">'append'</span> |
| <span class="k">continue</span> |
| <span class="k">elif</span> <span class="n">line</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">'Returns'</span><span class="p">):</span> |
| <span class="k">break</span> |
| |
| <span class="k">if</span> <span class="n">state</span> <span class="o">==</span> <span class="s1">'append'</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">skip</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">line</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">line</span><span class="p">[</span><span class="n">indent</span><span class="p">:]</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">' '</span><span class="p">):</span> |
| <span class="n">skip</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="n">line</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="n">arg</span> <span class="o">+</span> <span class="s1">' : '</span><span class="p">)</span> <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">exclude</span><span class="p">):</span> |
| <span class="n">skip</span> <span class="o">=</span> <span class="kc">True</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">skip</span><span class="p">:</span> |
| <span class="n">extra_lines</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">line</span><span class="p">[</span><span class="n">indent</span><span class="p">:])</span> |
| <span class="c1"># Expand title underline due to Parameters -> Pandas Parameters.</span> |
| <span class="n">extra_lines</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span> <span class="o">+=</span> <span class="s1">'-------'</span> |
| <span class="n">dest</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">+=</span> <span class="s1">'</span><span class="se">\n</span><span class="s1">'</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">extra_lines</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">dest</span> |
| |
| <span class="k">return</span> <span class="n">append</span> |
| |
| <div class="viewcode-block" id="ReadFromCsv"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadFromCsv">[docs]</a> |
| <span class="nd">@append_pandas_args</span><span class="p">(</span> |
| <span class="n">pandas</span><span class="o">.</span><span class="n">read_csv</span><span class="p">,</span> <span class="n">exclude</span><span class="o">=</span><span class="p">[</span><span class="s1">'filepath_or_buffer'</span><span class="p">,</span> <span class="s1">'iterator'</span><span class="p">])</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">ReadFromCsv</span><span class="p">(</span><span class="n">path</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> <span class="o">*</span><span class="p">,</span> <span class="n">splittable</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A PTransform for reading comma-separated values (csv) files into a</span> |
| <span class="sd"> PCollection.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> path (str): The file path to read from. The path can contain glob</span> |
| <span class="sd"> characters such as ``*`` and ``?``.</span> |
| <span class="sd"> splittable (bool): Whether the csv files are splittable at line</span> |
| <span class="sd"> boundaries, i.e. each line of this file represents a complete record.</span> |
| <span class="sd"> This should be set to False if single records span multiple lines (e.g.</span> |
| <span class="sd"> a quoted field has a newline inside of it). Setting this to false may</span> |
| <span class="sd"> disable liquid sharding.</span> |
| <span class="sd"> **kwargs: Extra arguments passed to `pandas.read_csv` (see below).</span> |
| <span class="sd"> """</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.dataframe.io</span><span class="w"> </span><span class="kn">import</span> <span class="n">ReadViaPandas</span> |
| <span class="k">return</span> <span class="s1">'ReadFromCsv'</span> <span class="o">>></span> <span class="n">ReadViaPandas</span><span class="p">(</span> |
| <span class="s1">'csv'</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">splittable</span><span class="o">=</span><span class="n">splittable</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| |
| <div class="viewcode-block" id="WriteToCsv"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.WriteToCsv">[docs]</a> |
| <span class="nd">@append_pandas_args</span><span class="p">(</span> |
| <span class="n">pandas</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_csv</span><span class="p">,</span> <span class="n">exclude</span><span class="o">=</span><span class="p">[</span><span class="s1">'path_or_buf'</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">,</span> <span class="s1">'index_label'</span><span class="p">])</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">WriteToCsv</span><span class="p">(</span> |
| <span class="n">path</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> |
| <span class="n">num_shards</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="n">file_naming</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="s1">'fileio.FileNaming'</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="c1"># pylint: disable=line-too-long</span> |
| |
| <span class="w"> </span><span class="sd">"""A PTransform for writing a schema'd PCollection as a (set of)</span> |
| <span class="sd"> comma-separated values (csv) files.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> path (str): The file path to write to. The files written will</span> |
| <span class="sd"> begin with this prefix, followed by a shard identifier (see</span> |
| <span class="sd"> `num_shards`) according to the `file_naming` parameter.</span> |
| <span class="sd"> num_shards (optional int): The number of shards to use in the distributed</span> |
| <span class="sd"> write. Defaults to None, letting the system choose an optimal value.</span> |
| <span class="sd"> file_naming (optional callable): A file-naming strategy, determining the</span> |
| <span class="sd"> actual shard names given their shard number, etc.</span> |
| <span class="sd"> See the section on `file naming</span> |
| <span class="sd"> <https://beam.apache.org/releases/pydoc/current/apache_beam.io.fileio.html#file-naming>`_</span> |
| <span class="sd"> Defaults to `fileio.default_file_naming`, which names files as</span> |
| <span class="sd"> `path-XXXXX-of-NNNNN`.</span> |
| <span class="sd"> **kwargs: Extra arguments passed to `pandas.Dataframe.to_csv` (see below).</span> |
| <span class="sd"> """</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.dataframe.io</span><span class="w"> </span><span class="kn">import</span> <span class="n">WriteViaPandas</span> |
| <span class="k">if</span> <span class="n">num_shards</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="s1">'num_shards'</span><span class="p">]</span> <span class="o">=</span> <span class="n">num_shards</span> |
| <span class="k">if</span> <span class="n">file_naming</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="s1">'file_naming'</span><span class="p">]</span> <span class="o">=</span> <span class="n">file_naming</span> |
| <span class="k">return</span> <span class="s1">'WriteToCsv'</span> <span class="o">>></span> <span class="n">WriteViaPandas</span><span class="p">(</span><span class="s1">'csv'</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| |
| <div class="viewcode-block" id="ReadFromJson"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.ReadFromJson">[docs]</a> |
| <span class="nd">@append_pandas_args</span><span class="p">(</span><span class="n">pandas</span><span class="o">.</span><span class="n">read_json</span><span class="p">,</span> <span class="n">exclude</span><span class="o">=</span><span class="p">[</span><span class="s1">'path_or_buf'</span><span class="p">])</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">ReadFromJson</span><span class="p">(</span> |
| <span class="n">path</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> |
| <span class="o">*</span><span class="p">,</span> |
| <span class="n">orient</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s1">'records'</span><span class="p">,</span> |
| <span class="n">lines</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span> |
| <span class="n">dtype</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">bool</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Any</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A PTransform for reading json values from files into a PCollection.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> path (str): The file path to read from. The path can contain glob</span> |
| <span class="sd"> characters such as ``*`` and ``?``.</span> |
| <span class="sd"> orient (str): Format of the json elements in the file.</span> |
| <span class="sd"> Default to 'records', meaning the file is expected to contain a list</span> |
| <span class="sd"> of json objects like `{field1: value1, field2: value2, ...}`.</span> |
| <span class="sd"> lines (bool): Whether each line should be considered a separate record,</span> |
| <span class="sd"> as opposed to the entire file being a valid JSON object or list.</span> |
| <span class="sd"> Defaults to True (unlike Pandas).</span> |
| <span class="sd"> dtype (bool): If True, infer dtypes; if a dict of column to dtype,</span> |
| <span class="sd"> then use those; if False, then don’t infer dtypes at all.</span> |
| <span class="sd"> Defaults to False (unlike Pandas).</span> |
| <span class="sd"> **kwargs: Extra arguments passed to `pandas.read_json` (see below).</span> |
| <span class="sd"> """</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.dataframe.io</span><span class="w"> </span><span class="kn">import</span> <span class="n">ReadViaPandas</span> |
| <span class="k">return</span> <span class="s1">'ReadFromJson'</span> <span class="o">>></span> <span class="n">ReadViaPandas</span><span class="p">(</span> |
| <span class="s1">'json'</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">orient</span><span class="o">=</span><span class="n">orient</span><span class="p">,</span> <span class="n">lines</span><span class="o">=</span><span class="n">lines</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="n">dtype</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| |
| <div class="viewcode-block" id="WriteToJson"> |
| <a class="viewcode-back" href="../../../apache_beam.io.textio.html#apache_beam.io.textio.WriteToJson">[docs]</a> |
| <span class="nd">@append_pandas_args</span><span class="p">(</span> |
| <span class="n">pandas</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_json</span><span class="p">,</span> <span class="n">exclude</span><span class="o">=</span><span class="p">[</span><span class="s1">'path_or_buf'</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">])</span> |
| <span class="k">def</span><span class="w"> </span><span class="nf">WriteToJson</span><span class="p">(</span> |
| <span class="n">path</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> |
| <span class="o">*</span><span class="p">,</span> |
| <span class="n">num_shards</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="n">file_naming</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="s1">'fileio.FileNaming'</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="n">orient</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s1">'records'</span><span class="p">,</span> |
| <span class="n">lines</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="c1"># pylint: disable=line-too-long</span> |
| |
| <span class="w"> </span><span class="sd">"""A PTransform for writing a PCollection as json values to files.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> path (str): The file path to write to. The files written will</span> |
| <span class="sd"> begin with this prefix, followed by a shard identifier (see</span> |
| <span class="sd"> `num_shards`) according to the `file_naming` parameter.</span> |
| <span class="sd"> num_shards (optional int): The number of shards to use in the distributed</span> |
| <span class="sd"> write. Defaults to None, letting the system choose an optimal value.</span> |
| <span class="sd"> file_naming (optional callable): A file-naming strategy, determining the</span> |
| <span class="sd"> actual shard names given their shard number, etc.</span> |
| <span class="sd"> See the section on `file naming</span> |
| <span class="sd"> <https://beam.apache.org/releases/pydoc/current/apache_beam.io.fileio.html#file-naming>`_</span> |
| <span class="sd"> Defaults to `fileio.default_file_naming`, which names files as</span> |
| <span class="sd"> `path-XXXXX-of-NNNNN`.</span> |
| <span class="sd"> orient (str): Format of the json elements in the file.</span> |
| <span class="sd"> Default to 'records', meaning the file will to contain a list</span> |
| <span class="sd"> of json objects like `{field1: value1, field2: value2, ...}`.</span> |
| <span class="sd"> lines (bool): Whether each line should be considered a separate record,</span> |
| <span class="sd"> as opposed to the entire file being a valid JSON object or list.</span> |
| <span class="sd"> Defaults to True if orient is 'records' (unlike Pandas).</span> |
| <span class="sd"> **kwargs: Extra arguments passed to `pandas.Dataframe.to_json`</span> |
| <span class="sd"> (see below).</span> |
| <span class="sd"> """</span> |
| <span class="kn">from</span><span class="w"> </span><span class="nn">apache_beam.dataframe.io</span><span class="w"> </span><span class="kn">import</span> <span class="n">WriteViaPandas</span> |
| <span class="k">if</span> <span class="n">num_shards</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="s1">'num_shards'</span><span class="p">]</span> <span class="o">=</span> <span class="n">num_shards</span> |
| <span class="k">if</span> <span class="n">file_naming</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="s1">'file_naming'</span><span class="p">]</span> <span class="o">=</span> <span class="n">file_naming</span> |
| <span class="k">if</span> <span class="n">lines</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">lines</span> <span class="o">=</span> <span class="n">orient</span> <span class="o">==</span> <span class="s1">'records'</span> |
| <span class="k">return</span> <span class="s1">'WriteToJson'</span> <span class="o">>></span> <span class="n">WriteViaPandas</span><span class="p">(</span> |
| <span class="s1">'json'</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">orient</span><span class="o">=</span><span class="n">orient</span><span class="p">,</span> <span class="n">lines</span><span class="o">=</span><span class="n">lines</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| |
| <span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span> |
| |
| <span class="k">def</span><span class="w"> </span><span class="nf">no_pandas</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">ImportError</span><span class="p">(</span><span class="s1">'Please install apache_beam[dataframe]'</span><span class="p">)</span> |
| |
| <span class="k">for</span> <span class="n">transform</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'ReadFromCsv'</span><span class="p">,</span> <span class="s1">'WriteToCsv'</span><span class="p">,</span> <span class="s1">'ReadFromJson'</span><span class="p">,</span> <span class="s1">'WriteToJson'</span><span class="p">):</span> |
| <span class="nb">globals</span><span class="p">()[</span><span class="n">transform</span><span class="p">]</span> <span class="o">=</span> <span class="n">no_pandas</span> |
| </pre></div> |
| |
| </div> |
| </div> |
| <footer> |
| |
| <hr/> |
| |
| <div role="contentinfo"> |
| <p>© Copyright %Y, Apache Beam.</p> |
| </div> |
| |
| Built with <a href="https://www.sphinx-doc.org/">Sphinx</a> using a |
| <a href="https://github.com/readthedocs/sphinx_rtd_theme">theme</a> |
| provided by <a href="https://readthedocs.org">Read the Docs</a>. |
| |
| |
| </footer> |
| </div> |
| </div> |
| </section> |
| </div> |
| <script> |
| jQuery(function () { |
| SphinxRtdTheme.Navigation.enable(true); |
| }); |
| </script> |
| |
| </body> |
| </html> |