blob: 97af6b05cd799f76b84a4b65caacce7b2bd91fec [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.dataframe.schemas &mdash; 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> &raquo;</li>
<li><a href="../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.dataframe.schemas</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.dataframe.schemas</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="sa">r</span><span class="sd">&quot;&quot;&quot;Utilities for relating schema-aware PCollections and dataframe transforms.</span>
<span class="sd">Imposes a mapping between native Python typings (specifically those compatible</span>
<span class="sd">with :mod:`apache_beam.typehints.schemas`), and common pandas dtypes::</span>
<span class="sd"> pandas dtype Python typing</span>
<span class="sd"> np.int{8,16,32,64} &lt;-----&gt; np.int{8,16,32,64}*</span>
<span class="sd"> pd.Int{8,16,32,64}Dtype &lt;-----&gt; Optional[np.int{8,16,32,64}]*</span>
<span class="sd"> np.float{32,64} &lt;-----&gt; Optional[np.float{32,64}]</span>
<span class="sd"> \--- np.float{32,64}</span>
<span class="sd"> Not supported &lt;------ Optional[bytes]</span>
<span class="sd"> np.bool &lt;-----&gt; np.bool</span>
<span class="sd"> np.dtype(&#39;S&#39;) &lt;-----&gt; bytes</span>
<span class="sd"> pd.BooleanDType() &lt;-----&gt; Optional[bool]</span>
<span class="sd"> pd.StringDType() &lt;-----&gt; Optional[str]</span>
<span class="sd"> \--- str</span>
<span class="sd"> np.object &lt;-----&gt; Any</span>
<span class="sd"> * int, float, bool are treated the same as np.int64, np.float64, np.bool</span>
<span class="sd">Note that when converting to pandas dtypes, any types not specified here are</span>
<span class="sd">shunted to ``np.object``.</span>
<span class="sd">Similarly when converting from pandas to Python types, types that aren&#39;t</span>
<span class="sd">otherwise specified here are shunted to ``Any``. Notably, this includes</span>
<span class="sd">``np.datetime64``.</span>
<span class="sd">Pandas does not support hierarchical data natively. Currently, all structured</span>
<span class="sd">types (``Sequence``, ``Mapping``, nested ``NamedTuple`` types), are</span>
<span class="sd">shunted to ``np.object`` like all other unknown types. In the future these</span>
<span class="sd">types may be given special consideration.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="c1"># pytype: skip-file</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">NamedTuple</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Optional</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TypeVar</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Union</span>
<span class="kn">import</span> <span class="nn">numpy</span> <span class="k">as</span> <span class="nn">np</span>
<span class="kn">import</span> <span class="nn">pandas</span> <span class="k">as</span> <span class="nn">pd</span>
<span class="kn">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</span>
<span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">typehints</span>
<span class="kn">from</span> <span class="nn">apache_beam.portability.api</span> <span class="kn">import</span> <span class="n">schema_pb2</span>
<span class="kn">from</span> <span class="nn">apache_beam.transforms.util</span> <span class="kn">import</span> <span class="n">BatchElements</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.native_type_compatibility</span> <span class="kn">import</span> <span class="n">_match_is_optional</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.schemas</span> <span class="kn">import</span> <span class="n">named_fields_from_element_type</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.schemas</span> <span class="kn">import</span> <span class="n">named_fields_to_schema</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.schemas</span> <span class="kn">import</span> <span class="n">named_tuple_from_schema</span>
<span class="kn">from</span> <span class="nn">apache_beam.typehints.schemas</span> <span class="kn">import</span> <span class="n">named_tuple_to_schema</span>
<span class="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="kn">import</span> <span class="n">proto_utils</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">(</span>
<span class="s1">&#39;BatchRowsAsDataFrame&#39;</span><span class="p">,</span>
<span class="s1">&#39;generate_proxy&#39;</span><span class="p">,</span>
<span class="s1">&#39;UnbatchPandas&#39;</span><span class="p">,</span>
<span class="s1">&#39;element_type_from_dataframe&#39;</span><span class="p">)</span>
<span class="n">T</span> <span class="o">=</span> <span class="n">TypeVar</span><span class="p">(</span><span class="s1">&#39;T&#39;</span><span class="p">,</span> <span class="n">bound</span><span class="o">=</span><span class="n">NamedTuple</span><span class="p">)</span>
<span class="c1"># Generate type map (presented visually in the docstring)</span>
<span class="n">_BIDIRECTIONAL</span> <span class="o">=</span> <span class="p">[</span>
<span class="p">(</span><span class="nb">bool</span><span class="p">,</span> <span class="nb">bool</span><span class="p">),</span>
<span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">int8</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">int8</span><span class="p">),</span>
<span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">int16</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">int16</span><span class="p">),</span>
<span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">int32</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">int32</span><span class="p">),</span>
<span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">int64</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">int64</span><span class="p">),</span>
<span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">Int8Dtype</span><span class="p">(),</span> <span class="n">Optional</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">int8</span><span class="p">]),</span>
<span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">Int16Dtype</span><span class="p">(),</span> <span class="n">Optional</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">int16</span><span class="p">]),</span>
<span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">Int32Dtype</span><span class="p">(),</span> <span class="n">Optional</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">int32</span><span class="p">]),</span>
<span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">Int64Dtype</span><span class="p">(),</span> <span class="n">Optional</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">int64</span><span class="p">]),</span>
<span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">float32</span><span class="p">,</span> <span class="n">Optional</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">float32</span><span class="p">]),</span>
<span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">float64</span><span class="p">,</span> <span class="n">Optional</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">float64</span><span class="p">]),</span>
<span class="p">(</span><span class="nb">object</span><span class="p">,</span> <span class="n">Any</span><span class="p">),</span>
<span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">StringDtype</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="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">BooleanDtype</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="p">]</span>
<span class="n">PANDAS_TO_BEAM</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">pd</span><span class="o">.</span><span class="n">Series</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">dtype</span><span class="p">:</span> <span class="n">fieldtype</span>
<span class="k">for</span> <span class="n">dtype</span><span class="p">,</span>
<span class="n">fieldtype</span> <span class="ow">in</span> <span class="n">_BIDIRECTIONAL</span>
<span class="p">}</span>
<span class="n">BEAM_TO_PANDAS</span> <span class="o">=</span> <span class="p">{</span><span class="n">fieldtype</span><span class="p">:</span> <span class="n">dtype</span> <span class="k">for</span> <span class="n">dtype</span><span class="p">,</span> <span class="n">fieldtype</span> <span class="ow">in</span> <span class="n">_BIDIRECTIONAL</span><span class="p">}</span>
<span class="c1"># Shunt non-nullable Beam types to the same pandas types as their non-nullable</span>
<span class="c1"># equivalents for FLOATs, DOUBLEs, and STRINGs. pandas has no non-nullable dtype</span>
<span class="c1"># for these.</span>
<span class="n">OPTIONAL_SHUNTS</span> <span class="o">=</span> <span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">float32</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">float64</span><span class="p">,</span> <span class="nb">str</span><span class="p">]</span>
<span class="k">for</span> <span class="n">typehint</span> <span class="ow">in</span> <span class="n">OPTIONAL_SHUNTS</span><span class="p">:</span>
<span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="n">typehint</span><span class="p">]</span> <span class="o">=</span> <span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="n">Optional</span><span class="p">[</span><span class="n">typehint</span><span class="p">]]</span>
<span class="c1"># int, float -&gt; int64, np.float64</span>
<span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">int64</span><span class="p">]</span>
<span class="n">BEAM_TO_PANDAS</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="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="n">Optional</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">int64</span><span class="p">]]</span>
<span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="nb">float</span><span class="p">]</span> <span class="o">=</span> <span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">float64</span><span class="p">]</span>
<span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="n">Optional</span><span class="p">[</span><span class="nb">float</span><span class="p">]]</span> <span class="o">=</span> <span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="n">Optional</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">float64</span><span class="p">]]</span>
<span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="nb">bytes</span><span class="p">]</span> <span class="o">=</span> <span class="s1">&#39;bytes&#39;</span>
<div class="viewcode-block" id="BatchRowsAsDataFrame"><a class="viewcode-back" href="../../../apache_beam.dataframe.schemas.html#apache_beam.dataframe.schemas.BatchRowsAsDataFrame">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">T</span><span class="p">)</span>
<span class="nd">@typehints</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">BatchRowsAsDataFrame</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A transform that batches schema-aware PCollection elements into DataFrames</span>
<span class="sd"> Batching parameters are inherited from</span>
<span class="sd"> :class:`~apache_beam.transforms.util.BatchElements`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="n">proxy</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="bp">self</span><span class="o">.</span><span class="n">_batch_elements_transform</span> <span class="o">=</span> <span class="n">BatchElements</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="bp">self</span><span class="o">.</span><span class="n">_proxy</span> <span class="o">=</span> <span class="n">proxy</span>
<div class="viewcode-block" id="BatchRowsAsDataFrame.expand"><a class="viewcode-back" href="../../../apache_beam.dataframe.schemas.html#apache_beam.dataframe.schemas.BatchRowsAsDataFrame.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="n">proxy</span> <span class="o">=</span> <span class="n">generate_proxy</span><span class="p">(</span>
<span class="n">pcoll</span><span class="o">.</span><span class="n">element_type</span><span class="p">)</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_proxy</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">_proxy</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">proxy</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">):</span>
<span class="n">columns</span> <span class="o">=</span> <span class="n">proxy</span><span class="o">.</span><span class="n">columns</span>
<span class="n">construct</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">batch</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">from_records</span><span class="p">(</span>
<span class="n">batch</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="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">proxy</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="n">dtype</span> <span class="o">=</span> <span class="n">proxy</span><span class="o">.</span><span class="n">dtype</span>
<span class="n">construct</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">batch</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">(</span><span class="n">batch</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="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;Unknown proxy type: </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="n">proxy</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pcoll</span> <span class="o">|</span> <span class="bp">self</span><span class="o">.</span><span class="n">_batch_elements_transform</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="n">construct</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="generate_proxy"><a class="viewcode-back" href="../../../apache_beam.dataframe.schemas.html#apache_beam.dataframe.schemas.generate_proxy">[docs]</a><span class="k">def</span> <span class="nf">generate_proxy</span><span class="p">(</span><span class="n">element_type</span><span class="p">):</span>
<span class="c1"># type: (type) -&gt; pd.DataFrame</span>
<span class="sd">&quot;&quot;&quot;Generate a proxy pandas object for the given PCollection element_type.</span>
<span class="sd"> Currently only supports generating a DataFrame proxy from a schema-aware</span>
<span class="sd"> PCollection or a Series proxy from a primitively typed PCollection.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">element_type</span> <span class="o">!=</span> <span class="n">Any</span> <span class="ow">and</span> <span class="n">element_type</span> <span class="ow">in</span> <span class="n">BEAM_TO_PANDAS</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">(</span><span class="n">dtype</span><span class="o">=</span><span class="n">BEAM_TO_PANDAS</span><span class="p">[</span><span class="n">element_type</span><span class="p">])</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">fields</span> <span class="o">=</span> <span class="n">named_fields_from_element_type</span><span class="p">(</span><span class="n">element_type</span><span class="p">)</span>
<span class="n">proxy</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">columns</span><span class="o">=</span><span class="p">[</span><span class="n">name</span> <span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">_</span> <span class="ow">in</span> <span class="n">fields</span><span class="p">])</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">typehint</span> <span class="ow">in</span> <span class="n">fields</span><span class="p">:</span>
<span class="c1"># Default to np.object. This is lossy, we won&#39;t be able to recover</span>
<span class="c1"># the type at the output.</span>
<span class="n">dtype</span> <span class="o">=</span> <span class="n">BEAM_TO_PANDAS</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">typehint</span><span class="p">,</span> <span class="nb">object</span><span class="p">)</span>
<span class="n">proxy</span><span class="p">[</span><span class="n">name</span><span class="p">]</span> <span class="o">=</span> <span class="n">proxy</span><span class="p">[</span><span class="n">name</span><span class="p">]</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span><span class="n">dtype</span><span class="p">)</span>
<span class="k">return</span> <span class="n">proxy</span></div>
<div class="viewcode-block" id="element_type_from_dataframe"><a class="viewcode-back" href="../../../apache_beam.dataframe.schemas.html#apache_beam.dataframe.schemas.element_type_from_dataframe">[docs]</a><span class="k">def</span> <span class="nf">element_type_from_dataframe</span><span class="p">(</span><span class="n">proxy</span><span class="p">,</span> <span class="n">include_indexes</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="c1"># type: (pd.DataFrame, bool) -&gt; type</span>
<span class="sd">&quot;&quot;&quot;Generate an element_type for an element-wise PCollection from a proxy</span>
<span class="sd"> pandas object. Currently only supports converting the element_type for</span>
<span class="sd"> a schema-aware PCollection to a proxy DataFrame.</span>
<span class="sd"> Currently only supports generating a DataFrame proxy from a schema-aware</span>
<span class="sd"> PCollection.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">output_columns</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">if</span> <span class="n">include_indexes</span><span class="p">:</span>
<span class="n">remaining_index_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">proxy</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="p">)</span>
<span class="n">i</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">while</span> <span class="nb">len</span><span class="p">(</span><span class="n">remaining_index_names</span><span class="p">):</span>
<span class="n">index_name</span> <span class="o">=</span> <span class="n">remaining_index_names</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span>
<span class="k">if</span> <span class="n">index_name</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Encountered an unnamed index. Cannot convert to a &quot;</span>
<span class="s2">&quot;schema-aware PCollection with include_indexes=True. &quot;</span>
<span class="s2">&quot;Please name all indexes or consider not including &quot;</span>
<span class="s2">&quot;indexes.&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">index_name</span> <span class="ow">in</span> <span class="n">remaining_index_names</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Encountered multiple indexes with the name &#39;</span><span class="si">%s</span><span class="s2">&#39;. &quot;</span>
<span class="s2">&quot;Cannot convert to a schema-aware PCollection with &quot;</span>
<span class="s2">&quot;include_indexes=True. Please ensure all indexes have &quot;</span>
<span class="s2">&quot;unique names or consider not including indexes.&quot;</span> <span class="o">%</span> <span class="n">index_name</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">index_name</span> <span class="ow">in</span> <span class="n">proxy</span><span class="o">.</span><span class="n">columns</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Encountered an index that has the same name as one &quot;</span>
<span class="s2">&quot;of the columns, &#39;</span><span class="si">%s</span><span class="s2">&#39;. Cannot convert to a &quot;</span>
<span class="s2">&quot;schema-aware PCollection with include_indexes=True. &quot;</span>
<span class="s2">&quot;Please ensure all indexes have unique names or &quot;</span>
<span class="s2">&quot;consider not including indexes.&quot;</span> <span class="o">%</span> <span class="n">index_name</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># its ok!</span>
<span class="n">output_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="p">(</span><span class="n">index_name</span><span class="p">,</span> <span class="n">proxy</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">get_level_values</span><span class="p">(</span><span class="n">i</span><span class="p">)</span><span class="o">.</span><span class="n">dtype</span><span class="p">))</span>
<span class="n">i</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="n">output_columns</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="nb">zip</span><span class="p">(</span><span class="n">proxy</span><span class="o">.</span><span class="n">columns</span><span class="p">,</span> <span class="n">proxy</span><span class="o">.</span><span class="n">dtypes</span><span class="p">))</span>
<span class="k">return</span> <span class="n">named_tuple_from_schema</span><span class="p">(</span>
<span class="n">named_fields_to_schema</span><span class="p">([(</span><span class="n">column</span><span class="p">,</span> <span class="n">_dtype_to_fieldtype</span><span class="p">(</span><span class="n">dtype</span><span class="p">))</span>
<span class="k">for</span> <span class="p">(</span><span class="n">column</span><span class="p">,</span> <span class="n">dtype</span><span class="p">)</span> <span class="ow">in</span> <span class="n">output_columns</span><span class="p">]))</span></div>
<span class="k">class</span> <span class="nc">_BaseDataframeUnbatchDoFn</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</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">namedtuple_ctor</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_namedtuple_ctor</span> <span class="o">=</span> <span class="n">namedtuple_ctor</span>
<span class="k">def</span> <span class="nf">_get_series</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">df</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">()</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">df</span><span class="p">):</span>
<span class="c1"># TODO: Only do null checks for nullable types</span>
<span class="k">def</span> <span class="nf">make_null_checking_generator</span><span class="p">(</span><span class="n">series</span><span class="p">):</span>
<span class="n">nulls</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">isnull</span><span class="p">(</span><span class="n">series</span><span class="p">)</span>
<span class="k">return</span> <span class="p">(</span><span class="kc">None</span> <span class="k">if</span> <span class="n">isnull</span> <span class="k">else</span> <span class="n">value</span> <span class="k">for</span> <span class="n">isnull</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">nulls</span><span class="p">,</span> <span class="n">series</span><span class="p">))</span>
<span class="n">all_series</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_series</span><span class="p">(</span><span class="n">df</span><span class="p">)</span>
<span class="n">iterators</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">make_null_checking_generator</span><span class="p">(</span><span class="n">series</span><span class="p">)</span> <span class="k">for</span> <span class="n">series</span><span class="p">,</span>
<span class="n">typehint</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">all_series</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_namedtuple_ctor</span><span class="o">.</span><span class="vm">__annotations__</span><span class="p">)</span>
<span class="p">]</span>
<span class="c1"># TODO: Avoid materializing the rows. Produce an object that references the</span>
<span class="c1"># underlying dataframe</span>
<span class="k">for</span> <span class="n">values</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">iterators</span><span class="p">):</span>
<span class="k">yield</span> <span class="bp">self</span><span class="o">.</span><span class="n">_namedtuple_ctor</span><span class="p">(</span><span class="o">*</span><span class="n">values</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">infer_output_type</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">input_type</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_namedtuple_ctor</span>
<span class="nd">@classmethod</span>
<span class="k">def</span> <span class="nf">_from_serialized_schema</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">schema_str</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">cls</span><span class="p">(</span>
<span class="n">named_tuple_from_schema</span><span class="p">(</span>
<span class="n">proto_utils</span><span class="o">.</span><span class="n">parse_Bytes</span><span class="p">(</span><span class="n">schema_str</span><span class="p">,</span> <span class="n">schema_pb2</span><span class="o">.</span><span class="n">Schema</span><span class="p">)))</span>
<span class="k">def</span> <span class="nf">__reduce__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># when pickling, use bytes representation of the schema.</span>
<span class="k">return</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_from_serialized_schema</span><span class="p">,</span>
<span class="p">(</span><span class="n">named_tuple_to_schema</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_namedtuple_ctor</span><span class="p">)</span><span class="o">.</span><span class="n">SerializeToString</span><span class="p">(),</span> <span class="p">))</span>
<span class="k">class</span> <span class="nc">_UnbatchNoIndex</span><span class="p">(</span><span class="n">_BaseDataframeUnbatchDoFn</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">_get_series</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">df</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span><span class="n">df</span><span class="p">[</span><span class="n">column</span><span class="p">]</span> <span class="k">for</span> <span class="n">column</span> <span class="ow">in</span> <span class="n">df</span><span class="o">.</span><span class="n">columns</span><span class="p">]</span>
<span class="k">class</span> <span class="nc">_UnbatchWithIndex</span><span class="p">(</span><span class="n">_BaseDataframeUnbatchDoFn</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">_get_series</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">df</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span><span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">get_level_values</span><span class="p">(</span><span class="n">i</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="nb">len</span><span class="p">(</span><span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="p">))</span>
<span class="p">]</span> <span class="o">+</span> <span class="p">[</span><span class="n">df</span><span class="p">[</span><span class="n">column</span><span class="p">]</span> <span class="k">for</span> <span class="n">column</span> <span class="ow">in</span> <span class="n">df</span><span class="o">.</span><span class="n">columns</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">_unbatch_transform</span><span class="p">(</span><span class="n">proxy</span><span class="p">,</span> <span class="n">include_indexes</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">proxy</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">):</span>
<span class="n">ctor</span> <span class="o">=</span> <span class="n">element_type_from_dataframe</span><span class="p">(</span><span class="n">proxy</span><span class="p">,</span> <span class="n">include_indexes</span><span class="o">=</span><span class="n">include_indexes</span><span class="p">)</span>
<span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span>
<span class="n">_UnbatchWithIndex</span><span class="p">(</span><span class="n">ctor</span><span class="p">)</span> <span class="k">if</span> <span class="n">include_indexes</span> <span class="k">else</span> <span class="n">_UnbatchNoIndex</span><span class="p">(</span><span class="n">ctor</span><span class="p">))</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">proxy</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="c1"># Raise a TypeError if proxy has an unknown type</span>
<span class="n">output_type</span> <span class="o">=</span> <span class="n">_dtype_to_fieldtype</span><span class="p">(</span><span class="n">proxy</span><span class="o">.</span><span class="n">dtype</span><span class="p">)</span>
<span class="c1"># TODO: Should the index ever be included for a Series?</span>
<span class="k">if</span> <span class="n">_match_is_optional</span><span class="p">(</span><span class="n">output_type</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">unbatch</span><span class="p">(</span><span class="n">series</span><span class="p">):</span>
<span class="k">for</span> <span class="n">isnull</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">isnull</span><span class="p">(</span><span class="n">series</span><span class="p">),</span> <span class="n">series</span><span class="p">):</span>
<span class="k">yield</span> <span class="kc">None</span> <span class="k">if</span> <span class="n">isnull</span> <span class="k">else</span> <span class="n">value</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">def</span> <span class="nf">unbatch</span><span class="p">(</span><span class="n">series</span><span class="p">):</span>
<span class="k">yield from</span> <span class="n">series</span>
<span class="k">return</span> <span class="n">beam</span><span class="o">.</span><span class="n">FlatMap</span><span class="p">(</span><span class="n">unbatch</span><span class="p">)</span><span class="o">.</span><span class="n">with_output_types</span><span class="p">(</span><span class="n">output_type</span><span class="p">)</span>
<span class="c1"># TODO: What about scalar inputs?</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Proxy &#39;</span><span class="si">%s</span><span class="s2">&#39; has unsupported type &#39;</span><span class="si">%s</span><span class="s2">&#39;&quot;</span> <span class="o">%</span> <span class="p">(</span><span class="n">proxy</span><span class="p">,</span> <span class="nb">type</span><span class="p">(</span><span class="n">proxy</span><span class="p">)))</span>
<span class="k">def</span> <span class="nf">_dtype_to_fieldtype</span><span class="p">(</span><span class="n">dtype</span><span class="p">):</span>
<span class="n">fieldtype</span> <span class="o">=</span> <span class="n">PANDAS_TO_BEAM</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">dtype</span><span class="p">)</span>
<span class="k">if</span> <span class="n">fieldtype</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="n">fieldtype</span>
<span class="k">elif</span> <span class="n">dtype</span><span class="o">.</span><span class="n">kind</span> <span class="o">==</span> <span class="s1">&#39;S&#39;</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">bytes</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">Any</span>
<div class="viewcode-block" id="UnbatchPandas"><a class="viewcode-back" href="../../../apache_beam.dataframe.schemas.html#apache_beam.dataframe.schemas.UnbatchPandas">[docs]</a><span class="nd">@typehints</span><span class="o">.</span><span class="n">with_input_types</span><span class="p">(</span><span class="n">Union</span><span class="p">[</span><span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">])</span>
<span class="k">class</span> <span class="nc">UnbatchPandas</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;A transform that explodes a PCollection of DataFrame or Series. DataFrame</span>
<span class="sd"> is converterd to a schema-aware PCollection, while Series is converted to its</span>
<span class="sd"> underlying type.</span>
<span class="sd"> Args:</span>
<span class="sd"> include_indexes: (optional, default: False) When unbatching a DataFrame</span>
<span class="sd"> if include_indexes=True, attempt to include index columns in the output</span>
<span class="sd"> schema for expanded DataFrames. Raises an error if any of the index</span>
<span class="sd"> levels are unnamed (name=None), or if any of the names are not unique</span>
<span class="sd"> among all column and index names.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">proxy</span><span class="p">,</span> <span class="n">include_indexes</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_proxy</span> <span class="o">=</span> <span class="n">proxy</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_include_indexes</span> <span class="o">=</span> <span class="n">include_indexes</span>
<div class="viewcode-block" id="UnbatchPandas.expand"><a class="viewcode-back" href="../../../apache_beam.dataframe.schemas.html#apache_beam.dataframe.schemas.UnbatchPandas.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">_unbatch_transform</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_proxy</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_include_indexes</span><span class="p">)</span></div></div>
</pre></div>
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/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>