| |
| |
| <!DOCTYPE html> |
| <!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]--> |
| <!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]--> |
| <head> |
| <meta charset="utf-8"> |
| |
| <meta name="viewport" content="width=device-width, initial-scale=1.0"> |
| |
| <title>apache_beam.io.parquetio — Apache Beam 2.38.0 documentation</title> |
| |
| |
| |
| |
| |
| |
| |
| |
| <script type="text/javascript" src="../../../_static/js/modernizr.min.js"></script> |
| |
| |
| <script type="text/javascript" id="documentation_options" data-url_root="../../../" src="../../../_static/documentation_options.js"></script> |
| <script type="text/javascript" src="../../../_static/jquery.js"></script> |
| <script type="text/javascript" src="../../../_static/underscore.js"></script> |
| <script type="text/javascript" src="../../../_static/doctools.js"></script> |
| <script type="text/javascript" src="../../../_static/language_data.js"></script> |
| <script async="async" type="text/javascript" src="https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.5/latest.js?config=TeX-AMS-MML_HTMLorMML"></script> |
| |
| <script type="text/javascript" src="../../../_static/js/theme.js"></script> |
| |
| |
| |
| |
| <link rel="stylesheet" href="../../../_static/css/theme.css" type="text/css" /> |
| <link rel="stylesheet" href="../../../_static/pygments.css" type="text/css" /> |
| <link rel="index" title="Index" href="../../../genindex.html" /> |
| <link rel="search" title="Search" href="../../../search.html" /> |
| </head> |
| |
| <body class="wy-body-for-nav"> |
| |
| |
| <div class="wy-grid-for-nav"> |
| |
| <nav data-toggle="wy-nav-shift" class="wy-nav-side"> |
| <div class="wy-side-scroll"> |
| <div class="wy-side-nav-search" > |
| |
| |
| |
| <a href="../../../index.html" class="icon icon-home"> Apache Beam |
| |
| |
| |
| </a> |
| |
| |
| |
| |
| <div class="version"> |
| 2.38.0 |
| </div> |
| |
| |
| |
| |
| <div role="search"> |
| <form id="rtd-search-form" class="wy-form" action="../../../search.html" method="get"> |
| <input type="text" name="q" placeholder="Search docs" /> |
| <input type="hidden" name="check_keywords" value="yes" /> |
| <input type="hidden" name="area" value="default" /> |
| </form> |
| </div> |
| |
| |
| </div> |
| |
| <div class="wy-menu wy-menu-vertical" data-spy="affix" role="navigation" aria-label="main navigation"> |
| |
| |
| |
| |
| |
| |
| <ul> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.coders.html">apache_beam.coders package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.dataframe.html">apache_beam.dataframe package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.io.html">apache_beam.io package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.metrics.html">apache_beam.metrics package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.ml.html">apache_beam.ml package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.options.html">apache_beam.options package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.portability.html">apache_beam.portability package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.runners.html">apache_beam.runners package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.transforms.html">apache_beam.transforms package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.typehints.html">apache_beam.typehints package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.utils.html">apache_beam.utils package</a></li> |
| </ul> |
| <ul> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.error.html">apache_beam.error module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li> |
| </ul> |
| |
| |
| |
| </div> |
| </div> |
| </nav> |
| |
| <section data-toggle="wy-nav-shift" class="wy-nav-content-wrap"> |
| |
| |
| <nav class="wy-nav-top" aria-label="top navigation"> |
| |
| <i data-toggle="wy-nav-top" class="fa fa-bars"></i> |
| <a href="../../../index.html">Apache Beam</a> |
| |
| </nav> |
| |
| |
| <div class="wy-nav-content"> |
| |
| <div class="rst-content"> |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| |
| <div role="navigation" aria-label="breadcrumbs navigation"> |
| |
| <ul class="wy-breadcrumbs"> |
| |
| <li><a href="../../../index.html">Docs</a> »</li> |
| |
| <li><a href="../../index.html">Module code</a> »</li> |
| |
| <li>apache_beam.io.parquetio</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.parquetio</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">"""``PTransforms`` for reading from and writing to Parquet files.</span> |
| |
| <span class="sd">Provides two read ``PTransform``\\s, ``ReadFromParquet`` and</span> |
| <span class="sd">``ReadAllFromParquet``, that produces a ``PCollection`` of records.</span> |
| <span class="sd">Each record of this ``PCollection`` will contain a single record read from</span> |
| <span class="sd">a Parquet file. Records that are of simple types will be mapped into</span> |
| <span class="sd">corresponding Python types. The actual parquet file operations are done by</span> |
| <span class="sd">pyarrow. Source splitting is supported at row group granularity.</span> |
| |
| <span class="sd">Additionally, this module provides a write ``PTransform`` ``WriteToParquet``</span> |
| <span class="sd">that can be used to write a given ``PCollection`` of Python objects to a</span> |
| <span class="sd">Parquet file.</span> |
| <span class="sd">"""</span> |
| <span class="c1"># pytype: skip-file</span> |
| |
| <span class="kn">from</span> <span class="nn">functools</span> <span class="kn">import</span> <span class="n">partial</span> |
| |
| <span class="kn">from</span> <span class="nn">apache_beam.io</span> <span class="kn">import</span> <span class="n">filebasedsink</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io</span> <span class="kn">import</span> <span class="n">filebasedsource</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io.filesystem</span> <span class="kn">import</span> <span class="n">CompressionTypes</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io.iobase</span> <span class="kn">import</span> <span class="n">RangeTracker</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io.iobase</span> <span class="kn">import</span> <span class="n">Read</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.io.iobase</span> <span class="kn">import</span> <span class="n">Write</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">DoFn</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">ParDo</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.transforms</span> <span class="kn">import</span> <span class="n">PTransform</span> |
| |
| <span class="k">try</span><span class="p">:</span> |
| <span class="kn">import</span> <span class="nn">pyarrow</span> <span class="k">as</span> <span class="nn">pa</span> |
| <span class="kn">import</span> <span class="nn">pyarrow.parquet</span> <span class="k">as</span> <span class="nn">pq</span> |
| <span class="k">except</span> <span class="ne">ImportError</span><span class="p">:</span> |
| <span class="n">pa</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="n">pq</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="n">ARROW_MAJOR_VERSION</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">ARROW_MAJOR_VERSION</span><span class="p">,</span> <span class="n">_</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="nb">map</span><span class="p">(</span><span class="nb">int</span><span class="p">,</span> <span class="n">pa</span><span class="o">.</span><span class="n">__version__</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">'.'</span><span class="p">))</span> |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'ReadFromParquet'</span><span class="p">,</span> |
| <span class="s1">'ReadAllFromParquet'</span><span class="p">,</span> |
| <span class="s1">'ReadFromParquetBatched'</span><span class="p">,</span> |
| <span class="s1">'ReadAllFromParquetBatched'</span><span class="p">,</span> |
| <span class="s1">'WriteToParquet'</span> |
| <span class="p">]</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_ArrowTableToRowDictionaries</span><span class="p">(</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="sd">""" A DoFn that consumes an Arrow table and yields a python dictionary for</span> |
| <span class="sd"> each row in the table."""</span> |
| <span class="k">def</span> <span class="nf">process</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">table</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="k">if</span> <span class="n">with_filename</span><span class="p">:</span> |
| <span class="n">file_name</span> <span class="o">=</span> <span class="n">table</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> |
| <span class="n">table</span> <span class="o">=</span> <span class="n">table</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span> |
| <span class="n">num_rows</span> <span class="o">=</span> <span class="n">table</span><span class="o">.</span><span class="n">num_rows</span> |
| <span class="n">data_items</span> <span class="o">=</span> <span class="n">table</span><span class="o">.</span><span class="n">to_pydict</span><span class="p">()</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">n</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">num_rows</span><span class="p">):</span> |
| <span class="n">row</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="k">for</span> <span class="n">column</span><span class="p">,</span> <span class="n">values</span> <span class="ow">in</span> <span class="n">data_items</span><span class="p">:</span> |
| <span class="n">row</span><span class="p">[</span><span class="n">column</span><span class="p">]</span> <span class="o">=</span> <span class="n">values</span><span class="p">[</span><span class="n">n</span><span class="p">]</span> |
| <span class="k">if</span> <span class="n">with_filename</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">row</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">row</span> |
| |
| |
| <div class="viewcode-block" id="ReadFromParquetBatched"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadFromParquetBatched">[docs]</a><span class="k">class</span> <span class="nc">ReadFromParquetBatched</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="sd">"""A :class:`~apache_beam.transforms.ptransform.PTransform` for reading</span> |
| <span class="sd"> Parquet files as a `PCollection` of `pyarrow.Table`. This `PTransform` is</span> |
| <span class="sd"> currently experimental. No backward-compatibility guarantees."""</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">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">validate</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="sd">""" Initializes :class:`~ReadFromParquetBatched`</span> |
| |
| <span class="sd"> An alternative to :class:`~ReadFromParquet` that yields each row group from</span> |
| <span class="sd"> the Parquet file as a `pyarrow.Table`. These Table instances can be</span> |
| <span class="sd"> processed directly, or converted to a pandas DataFrame for processing. For</span> |
| <span class="sd"> more information on supported types and schema, please see the pyarrow</span> |
| <span class="sd"> documentation.</span> |
| |
| <span class="sd"> .. testcode::</span> |
| |
| <span class="sd"> with beam.Pipeline() as p:</span> |
| <span class="sd"> dataframes = p \\</span> |
| <span class="sd"> | 'Read' >> beam.io.ReadFromParquetBatched('/mypath/mypqfiles*') \\</span> |
| <span class="sd"> | 'Convert to pandas' >> beam.Map(lambda table: table.to_pandas())</span> |
| |
| <span class="sd"> .. NOTE: We're not actually interested in this error; but if we get here,</span> |
| <span class="sd"> it means that the way of calling this transform hasn't changed.</span> |
| |
| <span class="sd"> .. testoutput::</span> |
| <span class="sd"> :hide:</span> |
| |
| <span class="sd"> Traceback (most recent call last):</span> |
| <span class="sd"> ...</span> |
| <span class="sd"> OSError: No files found based on the file pattern</span> |
| |
| <span class="sd"> See also: :class:`~ReadFromParquet`.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> file_pattern (str): the file glob to read</span> |
| <span class="sd"> min_bundle_size (int): the minimum size in bytes, to be considered when</span> |
| <span class="sd"> splitting the input into bundles.</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"> columns (List[str]): list of columns that will be read from files.</span> |
| <span class="sd"> A column name may be a prefix of a nested field, e.g. 'a' will select</span> |
| <span class="sd"> 'a.b', 'a.c', and 'a.d.e'</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="bp">self</span><span class="o">.</span><span class="n">_source</span> <span class="o">=</span> <span class="n">_create_parquet_source</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">validate</span><span class="o">=</span><span class="n">validate</span><span class="p">,</span> |
| <span class="n">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">,</span> |
| <span class="p">)</span> |
| |
| <div class="viewcode-block" id="ReadFromParquetBatched.expand"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadFromParquetBatched.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">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></div> |
| |
| <div class="viewcode-block" id="ReadFromParquetBatched.display_data"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadFromParquetBatched.display_data">[docs]</a> <span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">{</span><span class="s1">'source_dd'</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_source</span><span class="p">}</span></div></div> |
| |
| |
| <div class="viewcode-block" id="ReadFromParquet"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadFromParquet">[docs]</a><span class="k">class</span> <span class="nc">ReadFromParquet</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="sd">"""A :class:`~apache_beam.transforms.ptransform.PTransform` for reading</span> |
| <span class="sd"> Parquet files as a `PCollection` of dictionaries. This `PTransform` is</span> |
| <span class="sd"> currently experimental. No backward-compatibility guarantees."""</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">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">validate</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="sd">"""Initializes :class:`ReadFromParquet`.</span> |
| |
| <span class="sd"> Uses source ``_ParquetSource`` to read a set of Parquet files defined by</span> |
| <span class="sd"> a given file pattern.</span> |
| |
| <span class="sd"> If ``/mypath/myparquetfiles*`` is a file-pattern that points to a set of</span> |
| <span class="sd"> Parquet files, a :class:`~apache_beam.pvalue.PCollection` for the records in</span> |
| <span class="sd"> these Parquet files can be created in the following manner.</span> |
| |
| <span class="sd"> .. testcode::</span> |
| |
| <span class="sd"> with beam.Pipeline() as p:</span> |
| <span class="sd"> records = p | 'Read' >> beam.io.ReadFromParquet('/mypath/mypqfiles*')</span> |
| |
| <span class="sd"> .. NOTE: We're not actually interested in this error; but if we get here,</span> |
| <span class="sd"> it means that the way of calling this transform hasn't changed.</span> |
| |
| <span class="sd"> .. testoutput::</span> |
| <span class="sd"> :hide:</span> |
| |
| <span class="sd"> Traceback (most recent call last):</span> |
| <span class="sd"> ...</span> |
| <span class="sd"> OSError: No files found based on the file pattern</span> |
| |
| <span class="sd"> Each element of this :class:`~apache_beam.pvalue.PCollection` will contain</span> |
| <span class="sd"> a Python dictionary representing a single record. The keys will be of type</span> |
| <span class="sd"> :class:`str` and named after their corresponding column names. The values</span> |
| <span class="sd"> will be of the type defined in the corresponding Parquet schema. Records</span> |
| <span class="sd"> that are of simple types will be mapped into corresponding Python types.</span> |
| <span class="sd"> Records that are of complex types like list and struct will be mapped to</span> |
| <span class="sd"> Python list and dictionary respectively. For more information on supported</span> |
| <span class="sd"> types and schema, please see the pyarrow documentation.</span> |
| |
| <span class="sd"> See also: :class:`~ReadFromParquetBatched`.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> file_pattern (str): the file glob to read</span> |
| <span class="sd"> min_bundle_size (int): the minimum size in bytes, to be considered when</span> |
| <span class="sd"> splitting the input into bundles.</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"> columns (List[str]): list of columns that will be read from files.</span> |
| <span class="sd"> A column name may be a prefix of a nested field, e.g. 'a' will select</span> |
| <span class="sd"> 'a.b', 'a.c', and 'a.d.e'</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="bp">self</span><span class="o">.</span><span class="n">_source</span> <span class="o">=</span> <span class="n">_create_parquet_source</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">validate</span><span class="o">=</span><span class="n">validate</span><span class="p">,</span> |
| <span class="n">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">,</span> |
| <span class="p">)</span> |
| |
| <div class="viewcode-block" id="ReadFromParquet.expand"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadFromParquet.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalue</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pvalue</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">ParDo</span><span class="p">(</span><span class="n">_ArrowTableToRowDictionaries</span><span class="p">())</span></div> |
| |
| <div class="viewcode-block" id="ReadFromParquet.display_data"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadFromParquet.display_data">[docs]</a> <span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">{</span><span class="s1">'source_dd'</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_source</span><span class="p">}</span></div></div> |
| |
| |
| <div class="viewcode-block" id="ReadAllFromParquetBatched"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadAllFromParquetBatched">[docs]</a><span class="k">class</span> <span class="nc">ReadAllFromParquetBatched</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="sd">"""A ``PTransform`` for reading ``PCollection`` of Parquet files.</span> |
| |
| <span class="sd"> Uses source ``_ParquetSource`` to read a ``PCollection`` of Parquet files or</span> |
| <span class="sd"> file patterns and produce a ``PCollection`` of ``pyarrow.Table``, one for</span> |
| <span class="sd"> each Parquet file row group. This ``PTransform`` is currently experimental.</span> |
| <span class="sd"> No backward-compatibility guarantees.</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="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">columns</span><span class="o">=</span><span class="kc">None</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">label</span><span class="o">=</span><span class="s1">'ReadAllFiles'</span><span class="p">):</span> |
| <span class="sd">"""Initializes ``ReadAllFromParquet``.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> min_bundle_size: the minimum size in bytes, to be considered when</span> |
| <span class="sd"> splitting the input into bundles.</span> |
| <span class="sd"> desired_bundle_size: the desired size in bytes, to be considered when</span> |
| <span class="sd"> splitting the input into bundles.</span> |
| <span class="sd"> columns: list of columns that will be read from files. A column name</span> |
| <span class="sd"> may be a prefix of a nested field, e.g. 'a' will select</span> |
| <span class="sd"> 'a.b', 'a.c', and 'a.d.e'</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"> """</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">source_from_file</span> <span class="o">=</span> <span class="n">partial</span><span class="p">(</span> |
| <span class="n">_create_parquet_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">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_read_all_files</span> <span class="o">=</span> <span class="n">filebasedsource</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="n">CompressionTypes</span><span class="o">.</span><span class="n">UNCOMPRESSED</span><span class="p">,</span> |
| <span class="n">desired_bundle_size</span><span class="p">,</span> |
| <span class="n">min_bundle_size</span><span class="p">,</span> |
| <span class="n">source_from_file</span><span class="p">,</span> |
| <span class="n">with_filename</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">label</span> <span class="o">=</span> <span class="n">label</span> |
| |
| <div class="viewcode-block" id="ReadAllFromParquetBatched.expand"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadAllFromParquetBatched.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalue</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pvalue</span> <span class="o">|</span> <span class="bp">self</span><span class="o">.</span><span class="n">label</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="ReadAllFromParquet"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadAllFromParquet">[docs]</a><span class="k">class</span> <span class="nc">ReadAllFromParquet</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">with_filename</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="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_batches</span> <span class="o">=</span> <span class="n">ReadAllFromParquetBatched</span><span class="p">(</span> |
| <span class="n">with_filename</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_with_filename</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="ReadAllFromParquet.expand"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadAllFromParquet.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pvalue</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pvalue</span> <span class="o">|</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_batches</span> <span class="o">|</span> <span class="n">ParDo</span><span class="p">(</span> |
| <span class="n">_ArrowTableToRowDictionaries</span><span class="p">(),</span> <span class="n">with_filename</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_with_filename</span><span class="p">)</span></div></div> |
| |
| |
| <span class="k">def</span> <span class="nf">_create_parquet_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="mi">0</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">columns</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="k">return</span> \ |
| <span class="n">_ParquetSource</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">validate</span><span class="o">=</span><span class="n">validate</span><span class="p">,</span> |
| <span class="n">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">,</span> |
| <span class="p">)</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_ParquetUtils</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">find_first_row_group_index</span><span class="p">(</span><span class="n">pf</span><span class="p">,</span> <span class="n">start_offset</span><span class="p">):</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="n">_ParquetUtils</span><span class="o">.</span><span class="n">get_number_of_row_groups</span><span class="p">(</span><span class="n">pf</span><span class="p">)):</span> |
| <span class="n">row_group_start_offset</span> <span class="o">=</span> <span class="n">_ParquetUtils</span><span class="o">.</span><span class="n">get_offset</span><span class="p">(</span><span class="n">pf</span><span class="p">,</span> <span class="n">i</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">row_group_start_offset</span> <span class="o">>=</span> <span class="n">start_offset</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">i</span> |
| <span class="k">return</span> <span class="o">-</span><span class="mi">1</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">get_offset</span><span class="p">(</span><span class="n">pf</span><span class="p">,</span> <span class="n">row_group_index</span><span class="p">):</span> |
| <span class="n">first_column_metadata</span> <span class="o">=</span>\ |
| <span class="n">pf</span><span class="o">.</span><span class="n">metadata</span><span class="o">.</span><span class="n">row_group</span><span class="p">(</span><span class="n">row_group_index</span><span class="p">)</span><span class="o">.</span><span class="n">column</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">first_column_metadata</span><span class="o">.</span><span class="n">has_dictionary_page</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">first_column_metadata</span><span class="o">.</span><span class="n">dictionary_page_offset</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">first_column_metadata</span><span class="o">.</span><span class="n">data_page_offset</span> |
| |
| <span class="nd">@staticmethod</span> |
| <span class="k">def</span> <span class="nf">get_number_of_row_groups</span><span class="p">(</span><span class="n">pf</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pf</span><span class="o">.</span><span class="n">metadata</span><span class="o">.</span><span class="n">num_row_groups</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_ParquetSource</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="sd">"""A source for reading Parquet files.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">file_pattern</span><span class="p">,</span> <span class="n">min_bundle_size</span><span class="p">,</span> <span class="n">validate</span><span class="p">,</span> <span class="n">columns</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="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">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">_columns</span> <span class="o">=</span> <span class="n">columns</span> |
| |
| <span class="k">def</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">next_block_start</span> <span class="o">=</span> <span class="o">-</span><span class="mi">1</span> |
| |
| <span class="k">def</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">if</span> <span class="n">next_block_start</span> <span class="o">>=</span> <span class="n">stop_position</span><span class="p">:</span> |
| <span class="c1"># Next block starts at or after the suggested stop position. Hence</span> |
| <span class="c1"># there will not be split points to be claimed for the range ending at</span> |
| <span class="c1"># suggested stop position.</span> |
| <span class="k">return</span> <span class="mi">0</span> |
| <span class="k">return</span> <span class="n">RangeTracker</span><span class="o">.</span><span class="n">SPLIT_POINTS_UNKNOWN</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="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="k">if</span> <span class="n">start_offset</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">start_offset</span> <span class="o">=</span> <span class="mi">0</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">f</span><span class="p">:</span> |
| <span class="n">pf</span> <span class="o">=</span> <span class="n">pq</span><span class="o">.</span><span class="n">ParquetFile</span><span class="p">(</span><span class="n">f</span><span class="p">)</span> |
| |
| <span class="c1"># find the first dictionary page (or data page if there's no dictionary</span> |
| <span class="c1"># page available) offset after the given start_offset. This offset is also</span> |
| <span class="c1"># the starting offset of any row group since the Parquet specification</span> |
| <span class="c1"># describes that the data pages always come first before the meta data in</span> |
| <span class="c1"># each row group.</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="n">_ParquetUtils</span><span class="o">.</span><span class="n">find_first_row_group_index</span><span class="p">(</span><span class="n">pf</span><span class="p">,</span> <span class="n">start_offset</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">index</span> <span class="o">!=</span> <span class="o">-</span><span class="mi">1</span><span class="p">:</span> |
| <span class="n">next_block_start</span> <span class="o">=</span> <span class="n">_ParquetUtils</span><span class="o">.</span><span class="n">get_offset</span><span class="p">(</span><span class="n">pf</span><span class="p">,</span> <span class="n">index</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">next_block_start</span> <span class="o">=</span> <span class="n">range_tracker</span><span class="o">.</span><span class="n">stop_position</span><span class="p">()</span> |
| <span class="n">number_of_row_groups</span> <span class="o">=</span> <span class="n">_ParquetUtils</span><span class="o">.</span><span class="n">get_number_of_row_groups</span><span class="p">(</span><span class="n">pf</span><span class="p">)</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_block_start</span><span class="p">):</span> |
| <span class="n">table</span> <span class="o">=</span> <span class="n">pf</span><span class="o">.</span><span class="n">read_row_group</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_columns</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">index</span> <span class="o">+</span> <span class="mi">1</span> <span class="o"><</span> <span class="n">number_of_row_groups</span><span class="p">:</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="n">index</span> <span class="o">+</span> <span class="mi">1</span> |
| <span class="n">next_block_start</span> <span class="o">=</span> <span class="n">_ParquetUtils</span><span class="o">.</span><span class="n">get_offset</span><span class="p">(</span><span class="n">pf</span><span class="p">,</span> <span class="n">index</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">next_block_start</span> <span class="o">=</span> <span class="n">range_tracker</span><span class="o">.</span><span class="n">stop_position</span><span class="p">()</span> |
| |
| <span class="k">yield</span> <span class="n">table</span> |
| |
| |
| <div class="viewcode-block" id="WriteToParquet"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.WriteToParquet">[docs]</a><span class="k">class</span> <span class="nc">WriteToParquet</span><span class="p">(</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="sd">"""A ``PTransform`` for writing parquet files.</span> |
| |
| <span class="sd"> This ``PTransform`` is currently experimental. No backward-compatibility</span> |
| <span class="sd"> guarantees.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">file_path_prefix</span><span class="p">,</span> |
| <span class="n">schema</span><span class="p">,</span> |
| <span class="n">row_group_buffer_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="p">,</span> |
| <span class="n">record_batch_size</span><span class="o">=</span><span class="mi">1000</span><span class="p">,</span> |
| <span class="n">codec</span><span class="o">=</span><span class="s1">'none'</span><span class="p">,</span> |
| <span class="n">use_deprecated_int96_timestamps</span><span class="o">=</span><span class="kc">False</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">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">mime_type</span><span class="o">=</span><span class="s1">'application/x-parquet'</span><span class="p">):</span> |
| <span class="sd">"""Initialize a WriteToParquet transform.</span> |
| |
| <span class="sd"> Writes parquet files from a :class:`~apache_beam.pvalue.PCollection` of</span> |
| <span class="sd"> records. Each record is a dictionary with keys of a string type that</span> |
| <span class="sd"> represent column names. Schema must be specified like the example below.</span> |
| |
| <span class="sd"> .. testsetup::</span> |
| |
| <span class="sd"> from tempfile import NamedTemporaryFile</span> |
| <span class="sd"> import glob</span> |
| <span class="sd"> import os</span> |
| <span class="sd"> import pyarrow</span> |
| |
| <span class="sd"> filename = NamedTemporaryFile(delete=False).name</span> |
| |
| <span class="sd"> .. testcode::</span> |
| |
| <span class="sd"> with beam.Pipeline() as p:</span> |
| <span class="sd"> records = p | 'Read' >> beam.Create(</span> |
| <span class="sd"> [{'name': 'foo', 'age': 10}, {'name': 'bar', 'age': 20}]</span> |
| <span class="sd"> )</span> |
| <span class="sd"> _ = records | 'Write' >> beam.io.WriteToParquet(filename,</span> |
| <span class="sd"> pyarrow.schema(</span> |
| <span class="sd"> [('name', pyarrow.binary()), ('age', pyarrow.int64())]</span> |
| <span class="sd"> )</span> |
| <span class="sd"> )</span> |
| |
| <span class="sd"> .. testcleanup::</span> |
| |
| <span class="sd"> for output in glob.glob('{}*'.format(filename)):</span> |
| <span class="sd"> os.remove(output)</span> |
| |
| <span class="sd"> For more information on supported types and schema, please see the pyarrow</span> |
| <span class="sd"> document.</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"> schema: The schema to use, as type of ``pyarrow.Schema``.</span> |
| <span class="sd"> row_group_buffer_size: The byte size of the row group buffer. Note that</span> |
| <span class="sd"> this size is for uncompressed data on the memory and normally much</span> |
| <span class="sd"> bigger than the actual row group size written to a file.</span> |
| <span class="sd"> record_batch_size: The number of records in each record batch. Record</span> |
| <span class="sd"> batch is a basic unit used for storing data in the row group buffer.</span> |
| <span class="sd"> A higher record batch size implies low granularity on a row group buffer</span> |
| <span class="sd"> size. For configuring a row group size based on the number of records,</span> |
| <span class="sd"> set ``row_group_buffer_size`` to 1 and use ``record_batch_size`` to</span> |
| <span class="sd"> adjust the value.</span> |
| <span class="sd"> codec: The codec to use for block-level compression. Any string supported</span> |
| <span class="sd"> by the pyarrow specification is accepted.</span> |
| <span class="sd"> use_deprecated_int96_timestamps: Write nanosecond resolution timestamps to</span> |
| <span class="sd"> INT96 Parquet format. Defaults to False.</span> |
| <span class="sd"> file_name_suffix: Suffix for the files written.</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"> mime_type: The MIME type to use for the produced files, if the filesystem</span> |
| <span class="sd"> supports specifying MIME types.</span> |
| |
| <span class="sd"> Returns:</span> |
| <span class="sd"> A WriteToParquet transform 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="bp">self</span><span class="o">.</span><span class="n">_sink</span> <span class="o">=</span> \ |
| <span class="n">_create_parquet_sink</span><span class="p">(</span> |
| <span class="n">file_path_prefix</span><span class="p">,</span> |
| <span class="n">schema</span><span class="p">,</span> |
| <span class="n">codec</span><span class="p">,</span> |
| <span class="n">row_group_buffer_size</span><span class="p">,</span> |
| <span class="n">record_batch_size</span><span class="p">,</span> |
| <span class="n">use_deprecated_int96_timestamps</span><span class="p">,</span> |
| <span class="n">file_name_suffix</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">mime_type</span> |
| <span class="p">)</span> |
| |
| <div class="viewcode-block" id="WriteToParquet.expand"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.WriteToParquet.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span> |
| <span class="k">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 class="viewcode-block" id="WriteToParquet.display_data"><a class="viewcode-back" href="../../../apache_beam.io.parquetio.html#apache_beam.io.parquetio.WriteToParquet.display_data">[docs]</a> <span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">{</span><span class="s1">'sink_dd'</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">def</span> <span class="nf">_create_parquet_sink</span><span class="p">(</span> |
| <span class="n">file_path_prefix</span><span class="p">,</span> |
| <span class="n">schema</span><span class="p">,</span> |
| <span class="n">codec</span><span class="p">,</span> |
| <span class="n">row_group_buffer_size</span><span class="p">,</span> |
| <span class="n">record_batch_size</span><span class="p">,</span> |
| <span class="n">use_deprecated_int96_timestamps</span><span class="p">,</span> |
| <span class="n">file_name_suffix</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">mime_type</span><span class="p">):</span> |
| <span class="k">return</span> \ |
| <span class="n">_ParquetSink</span><span class="p">(</span> |
| <span class="n">file_path_prefix</span><span class="p">,</span> |
| <span class="n">schema</span><span class="p">,</span> |
| <span class="n">codec</span><span class="p">,</span> |
| <span class="n">row_group_buffer_size</span><span class="p">,</span> |
| <span class="n">record_batch_size</span><span class="p">,</span> |
| <span class="n">use_deprecated_int96_timestamps</span><span class="p">,</span> |
| <span class="n">file_name_suffix</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">mime_type</span> |
| <span class="p">)</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_ParquetSink</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="sd">"""A sink for parquet files."""</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">file_path_prefix</span><span class="p">,</span> |
| <span class="n">schema</span><span class="p">,</span> |
| <span class="n">codec</span><span class="p">,</span> |
| <span class="n">row_group_buffer_size</span><span class="p">,</span> |
| <span class="n">record_batch_size</span><span class="p">,</span> |
| <span class="n">use_deprecated_int96_timestamps</span><span class="p">,</span> |
| <span class="n">file_name_suffix</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">mime_type</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="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="kc">None</span><span class="p">,</span> |
| <span class="n">mime_type</span><span class="o">=</span><span class="n">mime_type</span><span class="p">,</span> |
| <span class="c1"># Compression happens at the block level using the supplied codec, and</span> |
| <span class="c1"># not at the file level.</span> |
| <span class="n">compression_type</span><span class="o">=</span><span class="n">CompressionTypes</span><span class="o">.</span><span class="n">UNCOMPRESSED</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_schema</span> <span class="o">=</span> <span class="n">schema</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_codec</span> <span class="o">=</span> <span class="n">codec</span> |
| <span class="k">if</span> <span class="n">ARROW_MAJOR_VERSION</span> <span class="o">==</span> <span class="mi">1</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_codec</span><span class="o">.</span><span class="n">lower</span><span class="p">()</span> <span class="o">==</span> <span class="s2">"lz4"</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"Due to ARROW-9424, writing with LZ4 compression is not supported in "</span> |
| <span class="s2">"pyarrow 1.x, please use a different pyarrow version or a different "</span> |
| <span class="sa">f</span><span class="s2">"codec. Your pyarrow version: </span><span class="si">{</span><span class="n">pa</span><span class="o">.</span><span class="n">__version__</span><span class="si">}</span><span class="s2">"</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_row_group_buffer_size</span> <span class="o">=</span> <span class="n">row_group_buffer_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_use_deprecated_int96_timestamps</span> <span class="o">=</span> <span class="n">use_deprecated_int96_timestamps</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span> <span class="o">=</span> <span class="p">[[]</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="nb">len</span><span class="p">(</span><span class="n">schema</span><span class="o">.</span><span class="n">names</span><span class="p">))]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_buffer_size</span> <span class="o">=</span> <span class="n">record_batch_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches_byte_size</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_file_handle</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="k">def</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="bp">self</span><span class="o">.</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">return</span> <span class="n">pq</span><span class="o">.</span><span class="n">ParquetWriter</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_file_handle</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_schema</span><span class="p">,</span> |
| <span class="n">compression</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_codec</span><span class="p">,</span> |
| <span class="n">use_deprecated_int96_timestamps</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_use_deprecated_int96_timestamps</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">write_record</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">writer</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="p">[</span><span class="mi">0</span><span class="p">])</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="bp">self</span><span class="o">.</span><span class="n">_flush_buffer</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches_byte_size</span> <span class="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_row_group_buffer_size</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_write_batches</span><span class="p">(</span><span class="n">writer</span><span class="p">)</span> |
| |
| <span class="c1"># reorder the data in columnar format.</span> |
| <span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">n</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_schema</span><span class="o">.</span><span class="n">names</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="p">[</span><span class="n">i</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value</span><span class="p">[</span><span class="n">n</span><span class="p">])</span> |
| |
| <span class="k">def</span> <span class="nf">close</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">writer</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_flush_buffer</span><span class="p">()</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches_byte_size</span> <span class="o">></span> <span class="mi">0</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_write_batches</span><span class="p">(</span><span class="n">writer</span><span class="p">)</span> |
| |
| <span class="n">writer</span><span class="o">.</span><span class="n">close</span><span class="p">()</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_file_handle</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_file_handle</span><span class="o">.</span><span class="n">close</span><span class="p">()</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_file_handle</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="k">def</span> <span class="nf">display_data</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="n">res</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">res</span><span class="p">[</span><span class="s1">'codec'</span><span class="p">]</span> <span class="o">=</span> <span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_codec</span><span class="p">)</span> |
| <span class="n">res</span><span class="p">[</span><span class="s1">'schema'</span><span class="p">]</span> <span class="o">=</span> <span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_schema</span><span class="p">)</span> |
| <span class="n">res</span><span class="p">[</span><span class="s1">'row_group_buffer_size'</span><span class="p">]</span> <span class="o">=</span> <span class="nb">str</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_row_group_buffer_size</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">res</span> |
| |
| <span class="k">def</span> <span class="nf">_write_batches</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">writer</span><span class="p">):</span> |
| <span class="n">table</span> <span class="o">=</span> <span class="n">pa</span><span class="o">.</span><span class="n">Table</span><span class="o">.</span><span class="n">from_batches</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_record_batches</span><span class="p">,</span> <span class="n">schema</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_schema</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches_byte_size</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="n">writer</span><span class="o">.</span><span class="n">write_table</span><span class="p">(</span><span class="n">table</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_flush_buffer</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="n">arrays</span> <span class="o">=</span> <span class="p">[[]</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="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_schema</span><span class="o">.</span><span class="n">names</span><span class="p">))]</span> |
| <span class="k">for</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="p">):</span> |
| <span class="n">arrays</span><span class="p">[</span><span class="n">x</span><span class="p">]</span> <span class="o">=</span> <span class="n">pa</span><span class="o">.</span><span class="n">array</span><span class="p">(</span><span class="n">y</span><span class="p">,</span> <span class="nb">type</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_schema</span><span class="o">.</span><span class="n">types</span><span class="p">[</span><span class="n">x</span><span class="p">])</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_buffer</span><span class="p">[</span><span class="n">x</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">rb</span> <span class="o">=</span> <span class="n">pa</span><span class="o">.</span><span class="n">RecordBatch</span><span class="o">.</span><span class="n">from_arrays</span><span class="p">(</span><span class="n">arrays</span><span class="p">,</span> <span class="n">schema</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_schema</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">rb</span><span class="p">)</span> |
| <span class="n">size</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">arrays</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">b</span> <span class="ow">in</span> <span class="n">x</span><span class="o">.</span><span class="n">buffers</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">b</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">size</span> <span class="o">=</span> <span class="n">size</span> <span class="o">+</span> <span class="n">b</span><span class="o">.</span><span class="n">size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches_byte_size</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_record_batches_byte_size</span> <span class="o">+</span> <span class="n">size</span> |
| </pre></div> |
| |
| </div> |
| |
| </div> |
| <footer> |
| |
| |
| <hr/> |
| |
| <div role="contentinfo"> |
| <p> |
| © Copyright |
| |
| </p> |
| </div> |
| Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/rtfd/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>. |
| |
| </footer> |
| |
| </div> |
| </div> |
| |
| </section> |
| |
| </div> |
| |
| |
| |
| <script type="text/javascript"> |
| jQuery(function () { |
| SphinxRtdTheme.Navigation.enable(true); |
| }); |
| </script> |
| |
| |
| |
| |
| |
| |
| </body> |
| </html> |