| |
| |
| <!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.frames — 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.dataframe.frames</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.frames</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="sd">"""Analogs for :class:`pandas.DataFrame` and :class:`pandas.Series`:</span> |
| <span class="sd">:class:`DeferredDataFrame` and :class:`DeferredSeries`.</span> |
| |
| <span class="sd">These classes are effectively wrappers around a `schema-aware`_</span> |
| <span class="sd">:class:`~apache_beam.pvalue.PCollection` that provide a set of operations</span> |
| <span class="sd">compatible with the `pandas`_ API.</span> |
| |
| <span class="sd">Note that we aim for the Beam DataFrame API to be completely compatible with</span> |
| <span class="sd">the pandas API, but there are some features that are currently unimplemented</span> |
| <span class="sd">for various reasons. Pay particular attention to the **'Differences from</span> |
| <span class="sd">pandas'** section for each operation to understand where we diverge.</span> |
| |
| <span class="sd">.. _schema-aware:</span> |
| <span class="sd"> https://beam.apache.org/documentation/programming-guide/#what-is-a-schema</span> |
| <span class="sd">.. _pandas:</span> |
| <span class="sd"> https://pandas.pydata.org/</span> |
| <span class="sd">"""</span> |
| |
| <span class="kn">import</span> <span class="nn">collections</span> |
| <span class="kn">import</span> <span class="nn">inspect</span> |
| <span class="kn">import</span> <span class="nn">itertools</span> |
| <span class="kn">import</span> <span class="nn">math</span> |
| <span class="kn">import</span> <span class="nn">re</span> |
| <span class="kn">import</span> <span class="nn">warnings</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">List</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Optional</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">from</span> <span class="nn">pandas._libs</span> <span class="kn">import</span> <span class="n">lib</span> |
| <span class="kn">from</span> <span class="nn">pandas.core.groupby.generic</span> <span class="kn">import</span> <span class="n">DataFrameGroupBy</span> |
| |
| <span class="kn">from</span> <span class="nn">apache_beam.dataframe</span> <span class="kn">import</span> <span class="n">expressions</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.dataframe</span> <span class="kn">import</span> <span class="n">frame_base</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.dataframe</span> <span class="kn">import</span> <span class="n">io</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.dataframe</span> <span class="kn">import</span> <span class="n">partitionings</span> |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'DeferredSeries'</span><span class="p">,</span> |
| <span class="s1">'DeferredDataFrame'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="c1"># Get major, minor version</span> |
| <span class="n">PD_VERSION</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="nb">int</span><span class="p">,</span> <span class="n">pd</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="mi">0</span><span class="p">:</span><span class="mi">2</span><span class="p">]))</span> |
| |
| |
| <span class="k">def</span> <span class="nf">populate_not_implemented</span><span class="p">(</span><span class="n">pd_type</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">wrapper</span><span class="p">(</span><span class="n">deferred_type</span><span class="p">):</span> |
| <span class="k">for</span> <span class="n">attr</span> <span class="ow">in</span> <span class="nb">dir</span><span class="p">(</span><span class="n">pd_type</span><span class="p">):</span> |
| <span class="c1"># Don't auto-define hidden methods or dunders</span> |
| <span class="k">if</span> <span class="n">attr</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">'_'</span><span class="p">):</span> |
| <span class="k">continue</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">deferred_type</span><span class="p">,</span> <span class="n">attr</span><span class="p">):</span> |
| <span class="n">pd_value</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">pd_type</span><span class="p">,</span> <span class="n">attr</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pd_value</span><span class="p">,</span> <span class="nb">property</span><span class="p">)</span> <span class="ow">or</span> <span class="n">inspect</span><span class="o">.</span><span class="n">isclass</span><span class="p">(</span><span class="n">pd_value</span><span class="p">):</span> |
| <span class="c1"># Some of the properties on pandas types (cat, dt, sparse), are</span> |
| <span class="c1"># actually attributes with class values, not properties</span> |
| <span class="nb">setattr</span><span class="p">(</span> |
| <span class="n">deferred_type</span><span class="p">,</span> |
| <span class="n">attr</span><span class="p">,</span> |
| <span class="nb">property</span><span class="p">(</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">not_implemented_method</span><span class="p">(</span><span class="n">attr</span><span class="p">,</span> <span class="n">base_type</span><span class="o">=</span><span class="n">pd_type</span><span class="p">)))</span> |
| <span class="k">elif</span> <span class="n">callable</span><span class="p">(</span><span class="n">pd_value</span><span class="p">):</span> |
| <span class="nb">setattr</span><span class="p">(</span> |
| <span class="n">deferred_type</span><span class="p">,</span> |
| <span class="n">attr</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">not_implemented_method</span><span class="p">(</span><span class="n">attr</span><span class="p">,</span> <span class="n">base_type</span><span class="o">=</span><span class="n">pd_type</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">deferred_type</span> |
| |
| <span class="k">return</span> <span class="n">wrapper</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_fillna_alias</span><span class="p">(</span><span class="n">method</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">wrapper</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="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">fillna</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="n">wrapper</span><span class="o">.</span><span class="vm">__name__</span> <span class="o">=</span> <span class="n">method</span> |
| <span class="n">wrapper</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="sa">f</span><span class="s1">'</span><span class="si">{</span><span class="n">method</span><span class="si">}</span><span class="s1"> is only supported for axis="columns". '</span> |
| <span class="s1">'axis="index" is order-sensitive.'</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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">frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">wrapper</span><span class="p">)))</span> |
| |
| |
| <span class="n">LIFTABLE_AGGREGATIONS</span> <span class="o">=</span> <span class="p">[</span><span class="s1">'all'</span><span class="p">,</span> <span class="s1">'any'</span><span class="p">,</span> <span class="s1">'max'</span><span class="p">,</span> <span class="s1">'min'</span><span class="p">,</span> <span class="s1">'prod'</span><span class="p">,</span> <span class="s1">'sum'</span><span class="p">]</span> |
| <span class="n">LIFTABLE_WITH_SUM_AGGREGATIONS</span> <span class="o">=</span> <span class="p">[</span><span class="s1">'size'</span><span class="p">,</span> <span class="s1">'count'</span><span class="p">]</span> |
| <span class="n">UNLIFTABLE_AGGREGATIONS</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'mean'</span><span class="p">,</span> |
| <span class="s1">'median'</span><span class="p">,</span> |
| <span class="s1">'quantile'</span><span class="p">,</span> |
| <span class="s1">'describe'</span><span class="p">,</span> |
| <span class="s1">'sem'</span><span class="p">,</span> |
| <span class="s1">'mad'</span><span class="p">,</span> |
| <span class="s1">'skew'</span><span class="p">,</span> |
| <span class="s1">'kurt'</span><span class="p">,</span> |
| <span class="s1">'kurtosis'</span><span class="p">,</span> |
| <span class="c1"># TODO: The below all have specialized distributed</span> |
| <span class="c1"># implementations, but they require tracking</span> |
| <span class="c1"># multiple intermediate series, which is difficult</span> |
| <span class="c1"># to lift in groupby</span> |
| <span class="s1">'std'</span><span class="p">,</span> |
| <span class="s1">'var'</span><span class="p">,</span> |
| <span class="s1">'corr'</span><span class="p">,</span> |
| <span class="s1">'cov'</span><span class="p">,</span> |
| <span class="s1">'nunique'</span><span class="p">,</span> |
| <span class="p">]</span> |
| <span class="n">ALL_AGGREGATIONS</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">LIFTABLE_AGGREGATIONS</span> <span class="o">+</span> <span class="n">LIFTABLE_WITH_SUM_AGGREGATIONS</span> <span class="o">+</span> |
| <span class="n">UNLIFTABLE_AGGREGATIONS</span><span class="p">)</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_agg_method</span><span class="p">(</span><span class="n">base</span><span class="p">,</span> <span class="n">func</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">wrapper</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="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">func</span> <span class="ow">in</span> <span class="n">UNLIFTABLE_AGGREGATIONS</span><span class="p">:</span> |
| <span class="n">wrapper</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"``</span><span class="si">{</span><span class="n">func</span><span class="si">}</span><span class="s2">`` cannot currently be parallelized. It will "</span> |
| <span class="s2">"require collecting all data on a single node."</span><span class="p">)</span> |
| <span class="n">wrapper</span><span class="o">.</span><span class="vm">__name__</span> <span class="o">=</span> <span class="n">func</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">base</span><span class="p">)(</span><span class="n">wrapper</span><span class="p">)</span> |
| |
| |
| <span class="c1"># Docstring to use for head and tail (commonly used to peek at datasets)</span> |
| <span class="n">_PEEK_METHOD_EXPLANATION</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s2">"because it is `order-sensitive "</span> |
| <span class="s2">"<https://s.apache.org/dataframe-order-sensitive-operations>`_.</span><span class="se">\n\n</span><span class="s2">"</span> |
| <span class="s2">"If you want to peek at a large dataset consider using interactive Beam's "</span> |
| <span class="s2">":func:`ib.collect "</span> |
| <span class="s2">"<apache_beam.runners.interactive.interactive_beam.collect>` "</span> |
| <span class="s2">"with ``n`` specified, or :meth:`sample`. If you want to find the "</span> |
| <span class="s2">"N largest elements, consider using :meth:`DeferredDataFrame.nlargest`."</span><span class="p">)</span> |
| |
| |
| <span class="k">class</span> <span class="nc">DeferredDataFrameOrSeries</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">_render_indexes</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="k">return</span> <span class="s1">'index='</span> <span class="o">+</span> <span class="p">(</span> |
| <span class="s1">'<unnamed>'</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="nb">repr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span><span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="s1">'indexes=['</span> <span class="o">+</span> <span class="s1">', '</span><span class="o">.</span><span class="n">join</span><span class="p">(</span> |
| <span class="s1">'<unnamed>'</span> <span class="k">if</span> <span class="n">ix</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="nb">repr</span><span class="p">(</span><span class="n">ix</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">ix</span> <span class="ow">in</span> <span class="bp">self</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="o">+</span> <span class="s1">']'</span> |
| |
| <span class="n">__array__</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'__array__'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">drop</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">labels</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">index</span><span class="p">,</span> <span class="n">columns</span><span class="p">,</span> <span class="n">errors</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""drop is not parallelizable when dropping from the index and</span> |
| <span class="sd"> ``errors="raise"`` is specified. It requires collecting all data on a single</span> |
| <span class="sd"> node in order to detect if one of the index values is missing."""</span> |
| <span class="k">if</span> <span class="n">labels</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</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">"Cannot specify both 'labels' and 'index'/'columns'"</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">):</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="n">labels</span> |
| <span class="n">columns</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">elif</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="n">columns</span> <span class="o">=</span> <span class="n">labels</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"axis must be one of (0, 1, 'index', 'columns'), "</span> |
| <span class="s2">"got '</span><span class="si">%s</span><span class="s2">'"</span> <span class="o">%</span> <span class="n">axis</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># Compute the proxy based on just the columns that are dropped.</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">drop</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="n">errors</span><span class="o">=</span><span class="n">errors</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">errors</span> <span class="o">==</span> <span class="s1">'raise'</span><span class="p">:</span> |
| <span class="c1"># In order to raise an error about missing index values, we'll</span> |
| <span class="c1"># need to collect the entire dataframe.</span> |
| <span class="c1"># TODO: This could be parallelized by putting index values in a</span> |
| <span class="c1"># ConstantExpression and partitioning by index.</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"drop(errors='raise', axis='index') is not currently "</span> |
| <span class="s2">"parallelizable. This requires collecting all data on a single "</span> |
| <span class="sa">f</span><span class="s2">"node in order to detect if one of </span><span class="si">{</span><span class="n">index</span><span class="si">!r}</span><span class="s2"> is missing."</span><span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'drop'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span> |
| <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> |
| <span class="n">index</span><span class="o">=</span><span class="n">index</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="n">errors</span><span class="o">=</span><span class="n">errors</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">droplevel</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">level</span><span class="p">,</span> <span class="n">axis</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'droplevel'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="n">level</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'column'</span><span class="p">)</span> <span class="k">else</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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">def</span> <span class="nf">swaplevel</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'swaplevel'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">swaplevel</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">fillna</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">,</span> <span class="n">method</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">limit</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""When ``axis="index"``, both ``method`` and ``limit`` must be ``None``.</span> |
| <span class="sd"> otherwise this operation is order-sensitive."""</span> |
| <span class="c1"># Default value is None, but is overriden with index.</span> |
| <span class="n">axis</span> <span class="o">=</span> <span class="n">axis</span> <span class="ow">or</span> <span class="s1">'index'</span> |
| |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">method</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"fillna(method=</span><span class="si">{</span><span class="n">method</span><span class="si">!r}</span><span class="s2">, axis=</span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">) is not supported "</span> |
| <span class="s2">"because it is order-sensitive. Only fillna(method=None) is "</span> |
| <span class="sa">f</span><span class="s2">"supported with axis=</span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">limit</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"fillna(limit=</span><span class="si">{</span><span class="n">method</span><span class="si">!r}</span><span class="s2">, axis=</span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">) is not supported because "</span> |
| <span class="s2">"it is order-sensitive. Only fillna(limit=None) is supported with "</span> |
| <span class="sa">f</span><span class="s2">"axis=</span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">DeferredDataFrame</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> |
| <span class="n">DeferredSeries</span><span class="p">):</span> |
| <span class="c1"># If self is a DataFrame and value is a Series we want to broadcast value</span> |
| <span class="c1"># to all partitions of self.</span> |
| <span class="c1"># This is OK, as its index must be the same size as the columns set of</span> |
| <span class="c1"># self, so cannot be too large.</span> |
| <span class="k">class</span> <span class="nc">AsScalar</span><span class="p">(</span><span class="nb">object</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">value</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">value</span> <span class="o">=</span> <span class="n">value</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">():</span> |
| <span class="n">value_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'as_scalar'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">AsScalar</span><span class="p">(</span><span class="n">df</span><span class="p">),</span> <span class="p">[</span><span class="n">value</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="n">get_value</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="o">.</span><span class="n">value</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="c1"># For other DeferredBase combinations, use Index partitioning to</span> |
| <span class="c1"># co-locate on the Index</span> |
| <span class="n">value_expr</span> <span class="o">=</span> <span class="n">value</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="n">get_value</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Default case, pass value through as a constant, no particular</span> |
| <span class="c1"># partitioning requirement</span> |
| <span class="n">value_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ConstantExpression</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> |
| <span class="n">get_value</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="c1"># yapf: disable</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'fillna'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> |
| <span class="n">value</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">fillna</span><span class="p">(</span> |
| <span class="n">get_value</span><span class="p">(</span><span class="n">value</span><span class="p">),</span> |
| <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> |
| <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> |
| <span class="n">limit</span><span class="o">=</span><span class="n">limit</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">value_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="nb">hasattr</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="s1">'ffill'</span><span class="p">):</span> |
| <span class="n">ffill</span> <span class="o">=</span> <span class="n">_fillna_alias</span><span class="p">(</span><span class="s1">'ffill'</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">hasattr</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="s1">'bfill'</span><span class="p">):</span> |
| <span class="n">bfill</span> <span class="o">=</span> <span class="n">_fillna_alias</span><span class="p">(</span><span class="s1">'bfill'</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">hasattr</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="s1">'backfill'</span><span class="p">):</span> |
| <span class="n">backfill</span> <span class="o">=</span> <span class="n">_fillna_alias</span><span class="p">(</span><span class="s1">'backfill'</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">hasattr</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="s1">'pad'</span><span class="p">):</span> |
| <span class="n">pad</span> <span class="o">=</span> <span class="n">_fillna_alias</span><span class="p">(</span><span class="s1">'pad'</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">first</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">offset</span><span class="p">):</span> |
| <span class="n">per_partition</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'first-per-partition'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">sort_index</span><span class="p">()</span><span class="o">.</span><span class="n">first</span><span class="p">(</span><span class="n">offset</span><span class="o">=</span><span class="n">offset</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'first'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">sort_index</span><span class="p">()</span><span class="o">.</span><span class="n">first</span><span class="p">(</span><span class="n">offset</span><span class="o">=</span><span class="n">offset</span><span class="p">),</span> <span class="p">[</span><span class="n">per_partition</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">last</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">offset</span><span class="p">):</span> |
| <span class="n">per_partition</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'last-per-partition'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">sort_index</span><span class="p">()</span><span class="o">.</span><span class="n">last</span><span class="p">(</span><span class="n">offset</span><span class="o">=</span><span class="n">offset</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'last'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">sort_index</span><span class="p">()</span><span class="o">.</span><span class="n">last</span><span class="p">(</span><span class="n">offset</span><span class="o">=</span><span class="n">offset</span><span class="p">),</span> <span class="p">[</span><span class="n">per_partition</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">groupby</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">by</span><span class="p">,</span> <span class="n">level</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">as_index</span><span class="p">,</span> <span class="n">group_keys</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``as_index`` and ``group_keys`` must both be ``True``.</span> |
| |
| <span class="sd"> Aggregations grouping by a categorical column with ``observed=False`` set</span> |
| <span class="sd"> are not currently parallelizable</span> |
| <span class="sd"> (`BEAM-11190 <https://issues.apache.org/jira/browse/BEAM-11190>`_).</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">as_index</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">'groupby(as_index=False)'</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">group_keys</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">'groupby(group_keys=False)'</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_DeferredGroupByCols</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'groupbycols'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">by</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">"You have to supply one of 'by' and 'level'"</span><span class="p">)</span> |
| |
| <span class="k">elif</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">level</span><span class="p">,</span> <span class="p">(</span><span class="nb">list</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">)):</span> |
| <span class="n">grouping_indexes</span> <span class="o">=</span> <span class="n">level</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">grouping_indexes</span> <span class="o">=</span> <span class="p">[</span><span class="n">level</span><span class="p">]</span> |
| |
| <span class="n">grouping_columns</span> <span class="o">=</span> <span class="p">[]</span> |
| |
| <span class="n">index</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span> |
| |
| <span class="c1"># Translate to level numbers only</span> |
| <span class="n">grouping_indexes</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">l</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">l</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="k">else</span> <span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">l</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">l</span> <span class="ow">in</span> <span class="n">grouping_indexes</span> |
| <span class="p">]</span> |
| |
| <span class="k">if</span> <span class="n">index</span><span class="o">.</span><span class="n">nlevels</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="n">to_group_with_index</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="n">to_group</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">levels_to_drop</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">i</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">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">)</span> <span class="k">if</span> <span class="n">i</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">grouping_indexes</span> |
| <span class="p">]</span> |
| |
| <span class="c1"># Reorder so the grouped indexes are first</span> |
| <span class="n">to_group_with_index</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">reorder_levels</span><span class="p">(</span> |
| <span class="n">grouping_indexes</span> <span class="o">+</span> <span class="n">levels_to_drop</span><span class="p">)</span> |
| |
| <span class="n">grouping_indexes</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">grouping_indexes</span><span class="p">)))</span> |
| <span class="n">levels_to_drop</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">grouping_indexes</span><span class="p">),</span> <span class="n">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">))</span> |
| <span class="k">if</span> <span class="n">levels_to_drop</span><span class="p">:</span> |
| <span class="n">to_group</span> <span class="o">=</span> <span class="n">to_group_with_index</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="n">levels_to_drop</span><span class="p">)</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">to_group</span> <span class="o">=</span> <span class="n">to_group_with_index</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="n">to_group_with_index</span> <span class="o">=</span> <span class="n">to_group_with_index</span><span class="o">.</span><span class="n">_expr</span> |
| |
| <span class="k">elif</span> <span class="n">callable</span><span class="p">(</span><span class="n">by</span><span class="p">):</span> |
| |
| <span class="k">def</span> <span class="nf">map_index</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">copy</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">df</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="n">by</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">df</span> |
| |
| <span class="n">to_group</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'map_index'</span><span class="p">,</span> |
| <span class="n">map_index</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="n">orig_nlevels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span> |
| |
| <span class="k">def</span> <span class="nf">prepend_mapped_index</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| |
| <span class="n">index</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span> |
| <span class="n">index</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="kc">None</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">map</span><span class="p">(</span><span class="n">by</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">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="o">.</span><span class="n">from_frame</span><span class="p">(</span> |
| <span class="n">index</span><span class="p">,</span> <span class="n">names</span><span class="o">=</span><span class="p">[</span><span class="kc">None</span><span class="p">]</span> <span class="o">+</span> <span class="nb">list</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="k">return</span> <span class="n">df</span> |
| |
| <span class="n">to_group_with_index</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'map_index_keep_orig'</span><span class="p">,</span> |
| <span class="n">prepend_mapped_index</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="c1"># Partitioning by the original indexes is preserved</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span> |
| <span class="nb">list</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="n">orig_nlevels</span> <span class="o">+</span> <span class="mi">1</span><span class="p">))))</span> |
| |
| <span class="n">grouping_columns</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="c1"># The index we need to group by is the last one</span> |
| <span class="n">grouping_indexes</span> <span class="o">=</span> <span class="p">[</span><span class="mi">0</span><span class="p">]</span> |
| |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">):</span> |
| |
| <span class="k">def</span> <span class="nf">set_index</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="n">by</span><span class="p">):</span> |
| <span class="n">df</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">s</span><span class="p">)</span> |
| <span class="n">df</span><span class="p">,</span> <span class="n">by</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">join</span><span class="o">=</span><span class="s1">'inner'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">by</span><span class="p">)</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:,</span> <span class="mi">0</span><span class="p">]</span> |
| |
| <span class="k">def</span> <span class="nf">prepend_index</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="n">by</span><span class="p">):</span> |
| <span class="n">df</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">s</span><span class="p">)</span> |
| <span class="n">df</span><span class="p">,</span> <span class="n">by</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">join</span><span class="o">=</span><span class="s1">'inner'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">set_index</span><span class="p">([</span><span class="n">by</span><span class="p">,</span> <span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="p">])</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:,</span> <span class="mi">0</span><span class="p">]</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| |
| <span class="k">def</span> <span class="nf">set_index</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="n">by</span><span class="p">):</span> <span class="c1"># type: ignore</span> |
| <span class="n">df</span><span class="p">,</span> <span class="n">by</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">join</span><span class="o">=</span><span class="s1">'inner'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">by</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">prepend_index</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="n">by</span><span class="p">):</span> <span class="c1"># type: ignore</span> |
| <span class="n">df</span><span class="p">,</span> <span class="n">by</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">join</span><span class="o">=</span><span class="s1">'inner'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">set_index</span><span class="p">([</span><span class="n">by</span><span class="p">,</span> <span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="p">])</span> |
| |
| <span class="n">to_group</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'set_index'</span><span class="p">,</span> |
| <span class="n">set_index</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">by</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="n">orig_nlevels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span> |
| <span class="n">to_group_with_index</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'prependindex'</span><span class="p">,</span> |
| <span class="n">prepend_index</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">by</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span> |
| <span class="nb">list</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="n">orig_nlevels</span> <span class="o">+</span> <span class="mi">1</span><span class="p">))))</span> |
| |
| <span class="n">grouping_columns</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">grouping_indexes</span> <span class="o">=</span> <span class="p">[</span><span class="mi">0</span><span class="p">]</span> |
| |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"Grouping by a concrete ndarray is order sensitive."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">DeferredDataFrame</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="n">by</span> <span class="o">=</span> <span class="p">[</span><span class="n">by</span><span class="p">]</span> |
| <span class="c1"># Find the columns that we need to move into the index so we can group by</span> |
| <span class="c1"># them</span> |
| <span class="n">column_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span> |
| <span class="n">grouping_columns</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">by</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="n">column_names</span><span class="p">))</span> |
| <span class="n">index_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span> |
| <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">by</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">index_names</span> <span class="ow">and</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> |
| <span class="n">grouping_indexes</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">by</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="n">index_names</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="n">grouping_indexes</span><span class="p">:</span> |
| <span class="k">if</span> <span class="nb">set</span><span class="p">(</span><span class="n">by</span><span class="p">)</span> <span class="o">==</span> <span class="nb">set</span><span class="p">(</span><span class="n">index_names</span><span class="p">):</span> |
| <span class="n">to_group</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="k">elif</span> <span class="nb">set</span><span class="p">(</span><span class="n">by</span><span class="p">)</span><span class="o">.</span><span class="n">issubset</span><span class="p">(</span><span class="n">index_names</span><span class="p">):</span> |
| <span class="n">to_group</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="n">index_names</span><span class="o">.</span><span class="n">difference</span><span class="p">(</span><span class="n">by</span><span class="p">))</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">to_group</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">grouping_indexes</span><span class="p">)</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">by</span><span class="p">)</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">to_group</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">by</span><span class="p">)</span><span class="o">.</span><span class="n">_expr</span> |
| |
| <span class="k">if</span> <span class="n">grouping_columns</span><span class="p">:</span> |
| <span class="c1"># TODO(BEAM-11711): It should be possible to do this without creating an</span> |
| <span class="c1"># expression manually, by using DeferredDataFrame.set_index, i.e.:</span> |
| <span class="c1"># to_group_with_index = self.set_index([self.index] +</span> |
| <span class="c1"># grouping_columns)._expr</span> |
| <span class="n">to_group_with_index</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'move_grouped_columns_to_index'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">set_index</span><span class="p">([</span><span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="p">]</span> <span class="o">+</span> <span class="n">grouping_columns</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="kc">False</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span> |
| <span class="nb">list</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">))))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">to_group_with_index</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</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="n">by</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">DeferredGroupBy</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'groupbyindex'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span> |
| <span class="n">level</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="nb">range</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">nlevels</span><span class="p">)),</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="n">to_group</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()),</span> |
| <span class="n">kwargs</span><span class="p">,</span> |
| <span class="n">to_group</span><span class="p">,</span> |
| <span class="n">to_group_with_index</span><span class="p">,</span> |
| <span class="n">grouping_columns</span><span class="o">=</span><span class="n">grouping_columns</span><span class="p">,</span> |
| <span class="n">grouping_indexes</span><span class="o">=</span><span class="n">grouping_indexes</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">loc</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_DeferredLoc</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">iloc</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="sd">"""Position-based indexing with `iloc` is order-sensitive in almost every</span> |
| <span class="sd"> case. Beam DataFrame users should prefer label-based indexing with `loc`.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="n">_DeferredILoc</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">reset_index</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">level</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="sd">"""Dropping the entire index (e.g. with ``reset_index(level=None)``) is</span> |
| <span class="sd"> not parallelizable. It is also only guaranteed that the newly generated</span> |
| <span class="sd"> index values will be unique. The Beam DataFrame API makes no guarantee</span> |
| <span class="sd"> that the same index values as the equivalent pandas operation will be</span> |
| <span class="sd"> generated, because that implementation is order-sensitive."""</span> |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">level</span><span class="p">,</span> <span class="p">(</span><span class="nb">tuple</span><span class="p">,</span> <span class="nb">list</span><span class="p">)):</span> |
| <span class="n">level</span> <span class="o">=</span> <span class="p">[</span><span class="n">level</span><span class="p">]</span> |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="nb">len</span><span class="p">(</span><span class="n">level</span><span class="p">)</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">:</span> |
| <span class="c1"># TODO(BEAM-12182): Could do distributed re-index with offsets.</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"reset_index(level=</span><span class="si">{</span><span class="n">level</span><span class="si">!r}</span><span class="s2">) drops the entire index and "</span> |
| <span class="s2">"creates a new one, so it cannot currently be parallelized "</span> |
| <span class="s2">"(BEAM-12182)."</span><span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'reset_index'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">level</span><span class="o">=</span><span class="n">level</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires_partition_by</span><span class="p">))</span> |
| |
| <span class="nb">abs</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'abs'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">astype</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">dtype</span><span class="p">,</span> <span class="n">copy</span><span class="p">,</span> <span class="n">errors</span><span class="p">):</span> |
| <span class="sd">"""astype is not parallelizable when ``errors="ignore"`` is specified.</span> |
| |
| <span class="sd"> ``copy=False`` is not supported because it relies on memory-sharing</span> |
| <span class="sd"> semantics.</span> |
| |
| <span class="sd"> ``dtype="category`` is not supported because the type of the output column</span> |
| <span class="sd"> depends on the data. Please use ``pd.CategoricalDtype`` with explicit</span> |
| <span class="sd"> categories instead.</span> |
| <span class="sd"> """</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="n">errors</span> <span class="o">==</span> <span class="s2">"ignore"</span><span class="p">:</span> |
| <span class="c1"># We need all data in order to ignore errors and propagate the original</span> |
| <span class="c1"># data.</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"astype(errors=</span><span class="si">{</span><span class="n">errors</span><span class="si">!r}</span><span class="s2">) is currently not parallelizable, "</span> |
| <span class="s2">"because all data must be collected on one node to determine if "</span> |
| <span class="s2">"the original data should be propagated instead."</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">copy</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"astype(copy=</span><span class="si">{</span><span class="n">copy</span><span class="si">!r}</span><span class="s2">) is not supported because it relies on "</span> |
| <span class="s2">"memory-sharing semantics that are not compatible with the Beam "</span> |
| <span class="s2">"model."</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">dtype</span> <span class="o">==</span> <span class="s1">'category'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"astype(dtype='category') is not supported because the type of the "</span> |
| <span class="s2">"output column depends on the data. Please use pd.CategoricalDtype "</span> |
| <span class="s2">"with explicit categories instead."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'astype'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">astype</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="n">copy</span><span class="o">=</span><span class="n">copy</span><span class="p">,</span> <span class="n">errors</span><span class="o">=</span><span class="n">errors</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="n">at_time</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span> |
| <span class="s1">'at_time'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">)</span> |
| <span class="n">between_time</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span> |
| <span class="s1">'between_time'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">)</span> |
| <span class="n">copy</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'copy'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">replace</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_replace</span><span class="p">,</span> <span class="n">value</span><span class="p">,</span> <span class="n">limit</span><span class="p">,</span> <span class="n">method</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``method`` is not supported in the Beam DataFrame API because it is</span> |
| <span class="sd"> order-sensitive. It cannot be specified.</span> |
| |
| <span class="sd"> If ``limit`` is specified this operation is not parallelizable."""</span> |
| <span class="n">value_compare</span> <span class="o">=</span> <span class="kc">None</span> <span class="k">if</span> <span class="n">PD_VERSION</span> <span class="o"><</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="mi">4</span><span class="p">)</span> <span class="k">else</span> <span class="n">lib</span><span class="o">.</span><span class="n">no_default</span> |
| <span class="k">if</span> <span class="n">method</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">to_replace</span><span class="p">,</span> |
| <span class="nb">dict</span><span class="p">)</span> <span class="ow">and</span> <span class="n">value</span> <span class="ow">is</span> <span class="n">value_compare</span><span class="p">:</span> |
| <span class="c1"># pandas only relies on method if to_replace is not a dictionary, and</span> |
| <span class="c1"># value is the <no_default> value. This is different than</span> |
| <span class="c1"># if ``None`` is explicitly passed for ``value``. In this case, it will be</span> |
| <span class="c1"># respected</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"replace(method=</span><span class="si">{</span><span class="n">method</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"order sensitive. Only replace(method=None) is supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">limit</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"replace(limit=</span><span class="si">{</span><span class="n">limit</span><span class="si">!r}</span><span class="s2">) cannot currently be parallelized. It "</span> |
| <span class="s2">"requires collecting all data on a single node."</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'replace'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span> |
| <span class="n">to_replace</span><span class="o">=</span><span class="n">to_replace</span><span class="p">,</span> |
| <span class="n">value</span><span class="o">=</span><span class="n">value</span><span class="p">,</span> |
| <span class="n">limit</span><span class="o">=</span><span class="n">limit</span><span class="p">,</span> |
| <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires_partition_by</span><span class="p">))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">tz_localize</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">ambiguous</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``ambiguous`` cannot be set to ``"infer"`` as its semantics are</span> |
| <span class="sd"> order-sensitive. Similarly, specifying ``ambiguous`` as an</span> |
| <span class="sd"> :class:`~numpy.ndarray` is order-sensitive, but you can achieve similar</span> |
| <span class="sd"> functionality by specifying ``ambiguous`` as a Series."""</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ambiguous</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"tz_localize(ambiguous=ndarray) is not supported because it makes "</span> |
| <span class="s2">"this operation sensitive to the order of the data. Please use a "</span> |
| <span class="s2">"DeferredSeries instead."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ambiguous</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'tz_localize'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> |
| <span class="n">ambiguous</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">tz_localize</span><span class="p">(</span><span class="n">ambiguous</span><span class="o">=</span><span class="n">ambiguous</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">ambiguous</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="n">ambiguous</span> <span class="o">==</span> <span class="s1">'infer'</span><span class="p">:</span> |
| <span class="c1"># infer attempts to infer based on the order of the timestamps</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"tz_localize(ambiguous=</span><span class="si">{</span><span class="n">ambiguous</span><span class="si">!r}</span><span class="s2">) is not allowed because it "</span> |
| <span class="s2">"makes this operation sensitive to the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'tz_localize'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">tz_localize</span><span class="p">(</span><span class="n">ambiguous</span><span class="o">=</span><span class="n">ambiguous</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">size</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="n">sizes</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'get_sizes'</span><span class="p">,</span> |
| <span class="c1"># Wrap scalar results in a Series for easier concatenation later</span> |
| <span class="k">lambda</span> <span class="n">df</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">df</span><span class="o">.</span><span class="n">size</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'sum_sizes'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">sizes</span><span class="p">:</span> <span class="n">sizes</span><span class="o">.</span><span class="n">sum</span><span class="p">(),</span> <span class="p">[</span><span class="n">sizes</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="k">def</span> <span class="nf">length</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="sd">"""Alternative to ``len(df)`` which returns a deferred result that can be</span> |
| <span class="sd"> used in arithmetic with :class:`DeferredSeries` or</span> |
| <span class="sd"> :class:`DeferredDataFrame` instances."""</span> |
| <span class="n">lengths</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'get_lengths'</span><span class="p">,</span> |
| <span class="c1"># Wrap scalar results in a Series for easier concatenation later</span> |
| <span class="k">lambda</span> <span class="n">df</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="nb">len</span><span class="p">(</span><span class="n">df</span><span class="p">)),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'sum_lengths'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">lengths</span><span class="p">:</span> <span class="n">lengths</span><span class="o">.</span><span class="n">sum</span><span class="p">(),</span> <span class="p">[</span><span class="n">lengths</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="k">def</span> <span class="fm">__len__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"len(df) is not currently supported because it produces a non-deferred "</span> |
| <span class="s2">"result. Consider using df.length() instead."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">empty</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="n">empties</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'get_empties'</span><span class="p">,</span> |
| <span class="c1"># Wrap scalar results in a Series for easier concatenation later</span> |
| <span class="k">lambda</span> <span class="n">df</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">df</span><span class="o">.</span><span class="n">empty</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'check_all_empty'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">empties</span><span class="p">:</span> <span class="n">empties</span><span class="o">.</span><span class="n">all</span><span class="p">(),</span> <span class="p">[</span><span class="n">empties</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">bool</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="c1"># TODO: Documentation about DeferredScalar</span> |
| <span class="c1"># Will throw if any partition has >1 element</span> |
| <span class="n">bools</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'get_bools'</span><span class="p">,</span> |
| <span class="c1"># Wrap scalar results in a Series for easier concatenation later</span> |
| <span class="k">lambda</span> <span class="n">df</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="nb">bool</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">df</span><span class="o">.</span><span class="n">empty</span> <span class="k">else</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">([</span><span class="n">df</span><span class="o">.</span><span class="n">bool</span><span class="p">()]),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="c1"># Will throw if overall dataset has != 1 element</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'combine_all_bools'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">bools</span><span class="p">:</span> <span class="n">bools</span><span class="o">.</span><span class="n">bool</span><span class="p">(),</span> <span class="p">[</span><span class="n">bools</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="nb">bool</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">equals</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span> |
| <span class="n">intermediate</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'equals_partitioned'</span><span class="p">,</span> |
| <span class="c1"># Wrap scalar results in a Series for easier concatenation later</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> |
| <span class="n">other</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">df</span><span class="o">.</span><span class="n">equals</span><span class="p">(</span><span class="n">other</span><span class="p">)),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'aggregate_equals'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">all</span><span class="p">(),</span> <span class="p">[</span><span class="n">intermediate</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">sort_values</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``sort_values`` is not implemented.</span> |
| |
| <span class="sd"> It is not implemented for ``axis=index`` because it imposes an ordering on</span> |
| <span class="sd"> the dataset, and it likely will not be maintained (see</span> |
| <span class="sd"> https://s.apache.org/dataframe-order-sensitive-operations).</span> |
| |
| <span class="sd"> It is not implemented for ``axis=columns`` because it makes the order of</span> |
| <span class="sd"> the columns depend on the data (see</span> |
| <span class="sd"> https://s.apache.org/dataframe-non-deferred-columns)."""</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">):</span> |
| <span class="c1"># axis=index imposes an ordering on the DataFrame rows which we do not</span> |
| <span class="c1"># support</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"sort_values(axis=index) is not supported because it imposes an "</span> |
| <span class="s2">"ordering on the dataset which likely will not be preserved."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># axis=columns will reorder the columns based on the data</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"sort_values(axis=columns) is not supported because the order of the "</span> |
| <span class="s2">"columns in the result depends on the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">sort_index</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``axis=index`` is not allowed because it imposes an ordering on the</span> |
| <span class="sd"> dataset, and we cannot guarantee it will be maintained (see</span> |
| <span class="sd"> https://s.apache.org/dataframe-order-sensitive-operations). Only</span> |
| <span class="sd"> ``axis=columns`` is allowed."""</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">):</span> |
| <span class="c1"># axis=rows imposes an ordering on the DataFrame which we do not support</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"sort_index(axis=index) is not supported because it imposes an "</span> |
| <span class="s2">"ordering on the dataset which we cannot guarantee will be "</span> |
| <span class="s2">"preserved."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="c1"># axis=columns reorders the columns by name</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'sort_index'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">sort_index</span><span class="p">(</span><span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="p">))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">where</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">cond</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">errors</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""where is not parallelizable when ``errors="ignore"`` is specified."""</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="n">deferred_args</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="n">actual_args</span> <span class="o">=</span> <span class="p">{}</span> |
| |
| <span class="c1"># TODO(bhulette): This is very similar to the logic in</span> |
| <span class="c1"># frame_base.elementwise_method, can we unify it?</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">cond</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="n">deferred_args</span><span class="p">[</span><span class="s1">'cond'</span><span class="p">]</span> <span class="o">=</span> <span class="n">cond</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">actual_args</span><span class="p">[</span><span class="s1">'cond'</span><span class="p">]</span> <span class="o">=</span> <span class="n">cond</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="n">deferred_args</span><span class="p">[</span><span class="s1">'other'</span><span class="p">]</span> <span class="o">=</span> <span class="n">other</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">actual_args</span><span class="p">[</span><span class="s1">'other'</span><span class="p">]</span> <span class="o">=</span> <span class="n">other</span> |
| |
| <span class="k">if</span> <span class="n">errors</span> <span class="o">==</span> <span class="s2">"ignore"</span><span class="p">:</span> |
| <span class="c1"># We need all data in order to ignore errors and propagate the original</span> |
| <span class="c1"># data.</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"where(errors=</span><span class="si">{</span><span class="n">errors</span><span class="si">!r}</span><span class="s2">) is currently not parallelizable, "</span> |
| <span class="s2">"because all data must be collected on one node to determine if "</span> |
| <span class="s2">"the original data should be propagated instead."</span><span class="p">))</span> |
| |
| <span class="n">actual_args</span><span class="p">[</span><span class="s1">'errors'</span><span class="p">]</span> <span class="o">=</span> <span class="n">errors</span> |
| |
| <span class="k">def</span> <span class="nf">where_execution</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">):</span> |
| <span class="n">runtime_values</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">name</span><span class="p">:</span> <span class="n">value</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">value</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">deferred_args</span><span class="o">.</span><span class="n">keys</span><span class="p">(),</span> <span class="n">args</span><span class="p">)</span> |
| <span class="p">}</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">where</span><span class="p">(</span><span class="o">**</span><span class="n">runtime_values</span><span class="p">,</span> <span class="o">**</span><span class="n">actual_args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s2">"where"</span><span class="p">,</span> |
| <span class="n">where_execution</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">]</span> <span class="o">+</span> <span class="p">[</span><span class="n">df</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">df</span> <span class="ow">in</span> <span class="n">deferred_args</span><span class="o">.</span><span class="n">values</span><span class="p">()],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="p">))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">mask</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">cond</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""mask is not parallelizable when ``errors="ignore"`` is specified."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">where</span><span class="p">(</span><span class="o">~</span><span class="n">cond</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">truncate</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">before</span><span class="p">,</span> <span class="n">after</span><span class="p">,</span> <span class="n">axis</span><span class="p">):</span> |
| |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">):</span> |
| |
| <span class="k">def</span> <span class="nf">truncate</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">sort_index</span><span class="p">()</span><span class="o">.</span><span class="n">truncate</span><span class="p">(</span><span class="n">before</span><span class="o">=</span><span class="n">before</span><span class="p">,</span> <span class="n">after</span><span class="o">=</span><span class="n">after</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| |
| <span class="k">def</span> <span class="nf">truncate</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">truncate</span><span class="p">(</span><span class="n">before</span><span class="o">=</span><span class="n">before</span><span class="p">,</span> <span class="n">after</span><span class="o">=</span><span class="n">after</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'truncate'</span><span class="p">,</span> |
| <span class="n">truncate</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">xs</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">level</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Note that ``xs(axis='index')`` will raise a ``KeyError`` at execution</span> |
| <span class="sd"> time if the key does not exist in the index."""</span> |
| |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'columns'</span><span class="p">,</span> <span class="mi">1</span><span class="p">):</span> |
| <span class="c1"># Special case for axis=columns. This is a simple project that raises a</span> |
| <span class="c1"># KeyError at construction time for missing columns.</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'xs'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">xs</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="n">axis</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'index'</span><span class="p">,</span> <span class="mi">0</span><span class="p">):</span> |
| <span class="c1"># Make sure that user's axis is valid</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"axis must be one of ('index', 0, 'columns', 1). "</span> |
| <span class="sa">f</span><span class="s2">"got </span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">."</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span> |
| <span class="n">key_size</span> <span class="o">=</span> <span class="mi">1</span> |
| <span class="n">key_series</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">([</span><span class="n">key</span><span class="p">],</span> <span class="n">index</span><span class="o">=</span><span class="p">[</span><span class="n">key</span><span class="p">])</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">key_size</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> |
| <span class="n">key_series</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">([</span><span class="n">key</span><span class="p">],</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="o">.</span><span class="n">from_tuples</span><span class="p">([</span><span class="n">key</span><span class="p">]))</span> |
| |
| <span class="n">key_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ConstantExpression</span><span class="p">(</span> |
| <span class="n">key_series</span><span class="p">,</span> <span class="n">proxy</span><span class="o">=</span><span class="n">key_series</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:</span><span class="mi">0</span><span class="p">])</span> |
| |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">reindexed</span> <span class="o">=</span> <span class="bp">self</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">level</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="n">level</span> <span class="o">=</span> <span class="p">[</span><span class="n">level</span><span class="p">]</span> |
| |
| <span class="c1"># If user specifed levels, reindex so those levels are at the beginning.</span> |
| <span class="c1"># Keep the others and preserve their order.</span> |
| <span class="n">level</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">l</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">l</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="k">else</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="p">)</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">l</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">l</span> <span class="ow">in</span> <span class="n">level</span> |
| <span class="p">]</span> |
| |
| <span class="n">reindexed</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">reorder_levels</span><span class="p">(</span> |
| <span class="n">level</span> <span class="o">+</span> <span class="p">[</span><span class="n">i</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="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">)</span> <span class="k">if</span> <span class="n">i</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">level</span><span class="p">])</span> |
| |
| <span class="k">def</span> <span class="nf">xs_partitioned</span><span class="p">(</span><span class="n">frame</span><span class="p">,</span> <span class="n">key</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">len</span><span class="p">(</span><span class="n">key</span><span class="p">):</span> |
| <span class="c1"># key is not in this partition, return empty dataframe</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">frame</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:</span><span class="mi">0</span><span class="p">]</span> |
| <span class="k">if</span> <span class="n">key_size</span> <span class="o"><</span> <span class="n">frame</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">result</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="n">key_size</span><span class="p">)))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">result</span> |
| |
| <span class="c1"># key should be in this partition, call xs. Will raise KeyError if not</span> |
| <span class="c1"># present.</span> |
| <span class="k">return</span> <span class="n">frame</span><span class="o">.</span><span class="n">xs</span><span class="p">(</span><span class="n">key</span><span class="o">.</span><span class="n">item</span><span class="p">())</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'xs'</span><span class="p">,</span> |
| <span class="n">xs_partitioned</span><span class="p">,</span> |
| <span class="p">[</span><span class="n">reindexed</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">key_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="n">key_size</span><span class="p">))),</span> |
| <span class="c1"># Drops index levels, so partitioning is not preserved</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">dtype</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dtype</span> |
| |
| <span class="n">isin</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'isin'</span><span class="p">,</span> <span class="n">base</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">combine_first</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span> |
| <span class="s1">'combine_first'</span><span class="p">,</span> <span class="n">base</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">combine</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="s1">'combine'</span><span class="p">,</span> |
| <span class="n">base</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">requires_partition_by</span><span class="o">=</span><span class="n">expressions</span><span class="o">.</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"combine() is not parallelizable because func might operate "</span> |
| <span class="s2">"on the full dataset."</span><span class="p">),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">expressions</span><span class="o">.</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">ndim</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">ndim</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">index</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_DeferredIndex</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> |
| |
| <span class="nd">@index</span><span class="o">.</span><span class="n">setter</span> |
| <span class="k">def</span> <span class="nf">_set_index</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="c1"># TODO: assigning the index is generally order-sensitive, but we could</span> |
| <span class="c1"># support it in some rare cases, e.g. when assigning the index from one</span> |
| <span class="c1"># of a DataFrame's columns</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="s2">"Assigning an index is not yet supported. "</span> |
| <span class="s2">"Consider using set_index() instead."</span><span class="p">)</span> |
| |
| <span class="n">reindex</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'reindex'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="n">hist</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'hist'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| |
| <span class="n">attrs</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'attrs'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'experimental'</span><span class="p">))</span> |
| |
| <span class="n">reorder_levels</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="s1">'reorder_levels'</span><span class="p">,</span> |
| <span class="n">base</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">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="n">resample</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'resample'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'event-time-semantics'</span><span class="p">)</span> |
| |
| <span class="n">rolling</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'rolling'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'event-time-semantics'</span><span class="p">)</span> |
| |
| <span class="n">to_xarray</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_xarray'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'non-deferred-result'</span><span class="p">)</span> |
| <span class="n">to_clipboard</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_clipboard'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="n">swapaxes</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'swapaxes'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| <span class="n">infer_object</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'infer_objects'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| |
| <span class="n">ewm</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'ewm'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"event-time-semantics"</span><span class="p">)</span> |
| <span class="n">expanding</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'expanding'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"event-time-semantics"</span><span class="p">)</span> |
| |
| <span class="n">sparse</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">not_implemented_method</span><span class="p">(</span> |
| <span class="s1">'sparse'</span><span class="p">,</span> <span class="s1">'BEAM-12425'</span><span class="p">,</span> <span class="n">base_type</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">transform</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'transform'</span><span class="p">,</span> <span class="n">base</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">tz_convert</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="s1">'tz_convert'</span><span class="p">,</span> |
| <span class="n">base</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">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="c1"># Manipulates index, partitioning is not preserved</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">pipe</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span> |
| <span class="n">func</span><span class="p">,</span> <span class="n">data</span> <span class="o">=</span> <span class="n">func</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="n">data</span><span class="p">]</span> <span class="o">=</span> <span class="bp">self</span> |
| <span class="k">return</span> <span class="n">func</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">func</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="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| |
| <div class="viewcode-block" id="DeferredSeries"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries">[docs]</a><span class="nd">@populate_not_implemented</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="nd">@frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">_register_for</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">DeferredSeries</span><span class="p">(</span><span class="n">DeferredDataFrameOrSeries</span><span class="p">):</span> |
| <span class="k">def</span> <span class="fm">__repr__</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="sa">f</span><span class="s1">'DeferredSeries(name=</span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">name</span><span class="si">!r}</span><span class="s1">, dtype=</span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">dtype</span><span class="si">}</span><span class="s1">, '</span> |
| <span class="sa">f</span><span class="s1">'</span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">_render_indexes</span><span class="p">()</span><span class="si">}</span><span class="s1">)'</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">name</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">name</span> |
| |
| <span class="nd">@name</span><span class="o">.</span><span class="n">setter</span> |
| <span class="k">def</span> <span class="nf">name</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">fn</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="n">s</span> <span class="o">=</span> <span class="n">s</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="n">s</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="n">value</span> |
| <span class="k">return</span> <span class="n">s</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'series_set_name'</span><span class="p">,</span> |
| <span class="n">fn</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">hasnans</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="n">has_nans</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'hasnans'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</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">s</span><span class="o">.</span><span class="n">hasnans</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">():</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'combine_hasnans'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">any</span><span class="p">(),</span> <span class="p">[</span><span class="n">has_nans</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">dtype</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dtype</span> |
| |
| <span class="n">dtypes</span> <span class="o">=</span> <span class="n">dtype</span> |
| |
| <span class="k">def</span> <span class="fm">__getitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">_is_null_slice</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="ow">or</span> <span class="n">key</span> <span class="ow">is</span> <span class="bp">Ellipsis</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span> |
| |
| <span class="k">elif</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="ow">or</span> <span class="n">_is_integer_slice</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> |
| <span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">_should_fallback_to_positional</span><span class="p">():</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"Accessing an item by an integer key is order sensitive for this "</span> |
| <span class="s2">"Series."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">slice</span><span class="p">)</span> <span class="ow">or</span> <span class="n">callable</span><span class="p">(</span><span class="n">key</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="c1"># yapf: disable</span> |
| <span class="s1">'getitem'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="p">[</span><span class="n">key</span><span class="p">],</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">)</span> <span class="ow">and</span> <span class="n">key</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dtype</span> <span class="o">==</span> <span class="nb">bool</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="c1"># yapf: disable</span> |
| <span class="s1">'getitem'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> |
| <span class="n">indexer</span><span class="p">:</span> <span class="n">df</span><span class="p">[</span><span class="n">indexer</span><span class="p">],</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">key</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="k">elif</span> <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">series</span><span class="o">.</span><span class="n">is_iterator</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="ow">or</span> <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">common</span><span class="o">.</span><span class="n">is_bool_indexer</span><span class="p">(</span><span class="n">key</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"Accessing a DeferredSeries with an iterator is sensitive to the "</span> |
| <span class="s2">"order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># We could consider returning a deferred scalar, but that might</span> |
| <span class="c1"># be more surprising than a clear error.</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"Indexing a series with key of type </span><span class="si">{</span><span class="nb">type</span><span class="p">(</span><span class="n">key</span><span class="p">)</span><span class="si">}</span><span class="s2"> is not supported "</span> |
| <span class="s2">"because it produces a non-deferred result."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.keys"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.keys">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">keys</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span></div> |
| |
| <span class="c1"># Series.T == transpose. Both are a no-op</span> |
| <span class="n">T</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'T'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| <span class="n">transpose</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'transpose'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| <span class="n">shape</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'shape'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">))</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.append"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.append">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">append</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_append</span><span class="p">,</span> <span class="n">ignore_index</span><span class="p">,</span> <span class="n">verify_integrity</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``ignore_index=True`` is not supported, because it requires generating an</span> |
| <span class="sd"> order-sensitive index."""</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">to_append</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"append() only accepts DeferredSeries instances, received "</span> <span class="o">+</span> |
| <span class="nb">str</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">to_append</span><span class="p">)))</span> |
| <span class="k">if</span> <span class="n">ignore_index</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"append(ignore_index=True) is order sensitive because it requires "</span> |
| <span class="s2">"generating a new index based on the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">verify_integrity</span><span class="p">:</span> |
| <span class="c1"># We can verify the index is non-unique within index partitioned data.</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'append'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">,</span> |
| <span class="n">to_append</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">append</span><span class="p">(</span> |
| <span class="n">to_append</span><span class="p">,</span> <span class="n">verify_integrity</span><span class="o">=</span><span class="n">verify_integrity</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">to_append</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.align"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.align">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">align</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">join</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">level</span><span class="p">,</span> <span class="n">method</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Aligning per-level is not yet supported. Only the default,</span> |
| <span class="sd"> ``level=None``, is allowed.</span> |
| |
| <span class="sd"> Filling NaN values via ``method`` is not supported, because it is</span> |
| <span class="sd"> `order-sensitive</span> |
| <span class="sd"> <https://s.apache.org/dataframe-order-sensitive-operations>`_.</span> |
| <span class="sd"> Only the default, ``method=None``, is allowed."""</span> |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">'per-level align'</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">method</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"align(method=</span><span class="si">{</span><span class="n">method</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"order sensitive. Only align(method=None) is supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="c1"># We're using pd.concat here as expressions don't yet support</span> |
| <span class="c1"># multiple return values.</span> |
| <span class="n">aligned</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'align'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> |
| <span class="n">y</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">concat</span><span class="p">([</span><span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">],</span> <span class="n">axis</span><span class="o">=</span><span class="mi">1</span><span class="p">,</span> <span class="n">join</span><span class="o">=</span><span class="s1">'inner'</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">return</span> <span class="n">aligned</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:,</span> <span class="mi">0</span><span class="p">],</span> <span class="n">aligned</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:,</span> <span class="mi">1</span><span class="p">]</span></div> |
| |
| <span class="n">argsort</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'argsort'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="n">array</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'array'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">))</span> |
| |
| <span class="c1"># We can't reliably predict the output type, it depends on whether `key` is:</span> |
| <span class="c1"># - not in the index (default_value)</span> |
| <span class="c1"># - in the index once (constant)</span> |
| <span class="c1"># - in the index multiple times (Series)</span> |
| <span class="n">get</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'get'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| |
| <span class="n">ravel</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'ravel'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="n">slice_shift</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'slice_shift'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"deprecated"</span><span class="p">)</span> |
| <span class="n">tshift</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'tshift'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"deprecated"</span><span class="p">)</span> |
| |
| <span class="n">rename</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="s1">'rename'</span><span class="p">,</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="n">between</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'between'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| |
| <span class="n">add_suffix</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="s1">'add_suffix'</span><span class="p">,</span> |
| <span class="n">base</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">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| <span class="n">add_prefix</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="s1">'add_prefix'</span><span class="p">,</span> |
| <span class="n">base</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">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="n">info</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'info'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_idxmaxmin_helper</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">op</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">op</span> <span class="o">==</span> <span class="s1">'idxmax'</span><span class="p">:</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="o">.</span><span class="n">idxmax</span> |
| <span class="k">elif</span> <span class="n">op</span> <span class="o">==</span> <span class="s1">'idxmin'</span><span class="p">:</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="o">.</span><span class="n">idxmin</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"op must be one of ('idxmax', 'idxmin'). "</span> |
| <span class="sa">f</span><span class="s2">"got </span><span class="si">{</span><span class="n">op</span><span class="si">!r}</span><span class="s2">."</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">compute_idx</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="n">func</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">pd</span><span class="o">.</span><span class="n">isna</span><span class="p">(</span><span class="n">index</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">s</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">s</span><span class="o">.</span><span class="n">loc</span><span class="p">[[</span><span class="n">index</span><span class="p">]]</span> |
| |
| <span class="c1"># Avoids empty Series error when evaluating proxy</span> |
| <span class="n">index_dtype</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">dtype</span> |
| <span class="n">index</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">([],</span> <span class="n">dtype</span><span class="o">=</span><span class="n">index_dtype</span><span class="p">)</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">copy</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">index</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="n">proxy</span><span class="o">.</span><span class="n">append</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="mi">1</span><span class="p">],</span> <span class="n">index</span><span class="o">=</span><span class="n">np</span><span class="o">.</span><span class="n">asarray</span><span class="p">([</span><span class="s1">'0'</span><span class="p">])</span><span class="o">.</span><span class="n">astype</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">dtype</span><span class="p">)))</span> |
| |
| <span class="n">idx_func</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'idx_func'</span><span class="p">,</span> |
| <span class="n">compute_idx</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'idx_combine'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">func</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="n">idx_func</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.idxmin"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.idxmin">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">idxmin</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_idxmaxmin_helper</span><span class="p">(</span><span class="s1">'idxmin'</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.idxmax"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.idxmax">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">idxmax</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_idxmaxmin_helper</span><span class="p">(</span><span class="s1">'idxmax'</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.explode"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.explode">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">explode</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">ignore_index</span><span class="p">):</span> |
| <span class="c1"># ignoring the index will not preserve it</span> |
| <span class="n">preserves</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()</span> <span class="k">if</span> <span class="n">ignore_index</span> <span class="k">else</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">())</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'explode'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span><span class="n">ignore_index</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">preserves</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.dot"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.dot">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">dot</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span> |
| <span class="sd">"""``other`` must be a :class:`DeferredDataFrame` or :class:`DeferredSeries`</span> |
| <span class="sd"> instance. Computing the dot product with an array-like is not supported</span> |
| <span class="sd"> because it is order-sensitive."""</span> |
| <span class="n">left</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">):</span> |
| <span class="n">right</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'to_dataframe'</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="p">[</span><span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="n">right_is_series</span> <span class="o">=</span> <span class="kc">True</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DeferredDataFrame</span><span class="p">):</span> |
| <span class="n">right</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="n">right_is_series</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"other must be a DeferredDataFrame or DeferredSeries instance. "</span> |
| <span class="s2">"Passing a concrete list or numpy array is not supported. Those "</span> |
| <span class="s2">"types have no index and must be joined based on the order of the "</span> |
| <span class="s2">"data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="n">dots</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'dot'</span><span class="p">,</span> |
| <span class="c1"># Transpose so we can sum across rows.</span> |
| <span class="p">(</span><span class="k">lambda</span> <span class="n">left</span><span class="p">,</span> <span class="n">right</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">left</span> <span class="o">@</span> <span class="n">right</span><span class="p">)</span><span class="o">.</span><span class="n">T</span><span class="p">),</span> |
| <span class="p">[</span><span class="n">left</span><span class="p">,</span> <span class="n">right</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="n">sums</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'sum'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">dots</span><span class="p">:</span> <span class="n">dots</span><span class="o">.</span><span class="n">sum</span><span class="p">(),</span> <span class="c1">#</span> |
| <span class="p">[</span><span class="n">dots</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="k">if</span> <span class="n">right_is_series</span><span class="p">:</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'extract'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="p">[</span><span class="n">sums</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">sums</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span><span class="n">result</span><span class="p">)</span></div> |
| |
| <span class="fm">__matmul__</span> <span class="o">=</span> <span class="n">dot</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.nunique"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.nunique">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">nunique</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">drop_duplicates</span><span class="p">(</span><span class="n">keep</span><span class="o">=</span><span class="s2">"any"</span><span class="p">)</span><span class="o">.</span><span class="n">size</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.quantile"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.quantile">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">quantile</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">q</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""quantile is not parallelizable. See</span> |
| <span class="sd"> `BEAM-12167 <https://issues.apache.org/jira/browse/BEAM-12167>`_ tracking</span> |
| <span class="sd"> the possible addition of an approximate, parallelizable implementation of</span> |
| <span class="sd"> quantile."""</span> |
| <span class="c1"># TODO(BEAM-12167): Provide an option for approximate distributed</span> |
| <span class="c1"># quantiles</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"Computing quantiles across index cannot currently be "</span> |
| <span class="s2">"parallelized. See BEAM-12167 tracking the possible addition of an "</span> |
| <span class="s2">"approximate, parallelizable implementation of quantile."</span><span class="p">))</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'quantile'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">quantile</span><span class="p">(</span><span class="n">q</span><span class="o">=</span><span class="n">q</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.std"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.std">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">std</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="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="c1"># Compute variance (deferred scalar) with same args, then sqrt it</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">var</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="o">.</span><span class="n">apply</span><span class="p">(</span><span class="k">lambda</span> <span class="n">var</span><span class="p">:</span> <span class="n">math</span><span class="o">.</span><span class="n">sqrt</span><span class="p">(</span><span class="n">var</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.var"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.var">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">var</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">skipna</span><span class="p">,</span> <span class="n">level</span><span class="p">,</span> <span class="n">ddof</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Per-level aggregation is not yet supported (BEAM-11777). Only the</span> |
| <span class="sd"> default, ``level=None``, is allowed."""</span> |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">"per-level aggregation"</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">skipna</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">skipna</span><span class="p">:</span> |
| <span class="bp">self</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">dropna</span><span class="p">()</span> <span class="c1"># pylint: disable=self-cls-assignment</span> |
| |
| <span class="c1"># See the online, numerically stable formulae at</span> |
| <span class="c1"># https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Parallel_algorithm</span> |
| <span class="c1"># and</span> |
| <span class="c1"># https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Welford's_online_algorithm</span> |
| <span class="k">def</span> <span class="nf">compute_moments</span><span class="p">(</span><span class="n">x</span><span class="p">):</span> |
| <span class="n">n</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> |
| <span class="n">m</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">std</span><span class="p">(</span><span class="n">ddof</span><span class="o">=</span><span class="mi">0</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="n">n</span> |
| <span class="n">s</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">sum</span><span class="p">()</span> |
| <span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="nb">dict</span><span class="p">(</span><span class="n">m</span><span class="o">=</span><span class="p">[</span><span class="n">m</span><span class="p">],</span> <span class="n">s</span><span class="o">=</span><span class="p">[</span><span class="n">s</span><span class="p">],</span> <span class="n">n</span><span class="o">=</span><span class="p">[</span><span class="n">n</span><span class="p">]))</span> |
| |
| <span class="k">def</span> <span class="nf">combine_moments</span><span class="p">(</span><span class="n">data</span><span class="p">):</span> |
| <span class="n">m</span> <span class="o">=</span> <span class="n">s</span> <span class="o">=</span> <span class="n">n</span> <span class="o">=</span> <span class="mf">0.0</span> |
| <span class="k">for</span> <span class="n">datum</span> <span class="ow">in</span> <span class="n">data</span><span class="o">.</span><span class="n">itertuples</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">continue</span> |
| <span class="k">elif</span> <span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">m</span><span class="p">,</span> <span class="n">s</span><span class="p">,</span> <span class="n">n</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">s</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">delta</span> <span class="o">=</span> <span class="n">s</span> <span class="o">/</span> <span class="n">n</span> <span class="o">-</span> <span class="n">datum</span><span class="o">.</span><span class="n">s</span> <span class="o">/</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> |
| <span class="n">m</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m</span> <span class="o">+</span> <span class="n">delta</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="n">n</span> <span class="o">*</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> <span class="o">/</span> <span class="p">(</span><span class="n">n</span> <span class="o">+</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span><span class="p">)</span> |
| <span class="n">s</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">s</span> |
| <span class="n">n</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> |
| <span class="k">if</span> <span class="n">n</span> <span class="o"><=</span> <span class="n">ddof</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">float</span><span class="p">(</span><span class="s1">'nan'</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">m</span> <span class="o">/</span> <span class="p">(</span><span class="n">n</span> <span class="o">-</span> <span class="n">ddof</span><span class="p">)</span> |
| |
| <span class="n">moments</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'compute_moments'</span><span class="p">,</span> |
| <span class="n">compute_moments</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'combine_moments'</span><span class="p">,</span> |
| <span class="n">combine_moments</span><span class="p">,</span> <span class="p">[</span><span class="n">moments</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.corr"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.corr">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">corr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">method</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">):</span> |
| <span class="sd">"""Only ``method='pearson'`` is currently parallelizable."""</span> |
| <span class="k">if</span> <span class="n">method</span> <span class="o">==</span> <span class="s1">'pearson'</span><span class="p">:</span> <span class="c1"># Note that this is the default.</span> |
| <span class="n">x</span><span class="p">,</span> <span class="n">y</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">dropna</span><span class="p">()</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">dropna</span><span class="p">(),</span> <span class="s1">'inner'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">x</span><span class="o">.</span><span class="n">_corr_aligned</span><span class="p">(</span><span class="n">y</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">)</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">reason</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"Encountered corr(method=</span><span class="si">{</span><span class="n">method</span><span class="si">!r}</span><span class="s2">) which cannot be "</span> |
| <span class="s2">"parallelized. Only corr(method='pearson') is currently "</span> |
| <span class="s2">"parallelizable."</span><span class="p">)</span> |
| <span class="c1"># The rank-based correlations are not obviously parallelizable, though</span> |
| <span class="c1"># perhaps an approximation could be done with a knowledge of quantiles</span> |
| <span class="c1"># and custom partitioning.</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'corr'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> |
| <span class="n">other</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">corr</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="n">min_periods</span><span class="o">=</span><span class="n">min_periods</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="n">reason</span><span class="p">)))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.skew"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.skew">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">skew</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">skipna</span><span class="p">,</span> <span class="n">level</span><span class="p">,</span> <span class="n">numeric_only</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">"per-level aggregation"</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">skipna</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">skipna</span><span class="p">:</span> |
| <span class="bp">self</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">dropna</span><span class="p">()</span> <span class="c1"># pylint: disable=self-cls-assignment</span> |
| <span class="c1"># See the online, numerically stable formulae at</span> |
| <span class="c1"># https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics</span> |
| <span class="c1"># Note that we are calculating the unbias (sample) version of skew here.</span> |
| <span class="c1"># See https://en.wikipedia.org/wiki/Skewness#Sample_skewness</span> |
| <span class="c1"># for more details.</span> |
| <span class="k">def</span> <span class="nf">compute_moments</span><span class="p">(</span><span class="n">x</span><span class="p">):</span> |
| <span class="n">n</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">m2</span><span class="p">,</span> <span class="nb">sum</span><span class="p">,</span> <span class="n">m3</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="mi">0</span><span class="p">,</span> <span class="mi">0</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">m2</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">std</span><span class="p">(</span><span class="n">ddof</span><span class="o">=</span><span class="mi">0</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="n">n</span> |
| <span class="nb">sum</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">sum</span><span class="p">()</span> |
| <span class="n">m3</span> <span class="o">=</span> <span class="p">(((</span><span class="n">x</span> <span class="o">-</span> <span class="n">x</span><span class="o">.</span><span class="n">mean</span><span class="p">())</span><span class="o">**</span><span class="mi">3</span><span class="p">)</span><span class="o">.</span><span class="n">sum</span><span class="p">())</span> |
| <span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="nb">dict</span><span class="p">(</span><span class="n">m2</span><span class="o">=</span><span class="p">[</span><span class="n">m2</span><span class="p">],</span> <span class="nb">sum</span><span class="o">=</span><span class="p">[</span><span class="nb">sum</span><span class="p">],</span> <span class="n">n</span><span class="o">=</span><span class="p">[</span><span class="n">n</span><span class="p">],</span> <span class="n">m3</span><span class="o">=</span><span class="p">[</span><span class="n">m3</span><span class="p">]))</span> |
| |
| <span class="k">def</span> <span class="nf">combine_moments</span><span class="p">(</span><span class="n">data</span><span class="p">):</span> |
| <span class="n">m2</span> <span class="o">=</span> <span class="nb">sum</span> <span class="o">=</span> <span class="n">n</span> <span class="o">=</span> <span class="n">m3</span> <span class="o">=</span> <span class="mf">0.0</span> |
| <span class="k">for</span> <span class="n">datum</span> <span class="ow">in</span> <span class="n">data</span><span class="o">.</span><span class="n">itertuples</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">continue</span> |
| <span class="k">elif</span> <span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">m2</span><span class="p">,</span> <span class="nb">sum</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">m3</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m2</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">sum</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">m3</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">n_a</span><span class="p">,</span> <span class="n">n_b</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span><span class="p">,</span> <span class="n">n</span> |
| <span class="n">sum_a</span><span class="p">,</span> <span class="n">sum_b</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">sum</span><span class="p">,</span> <span class="nb">sum</span> |
| <span class="n">m2_a</span><span class="p">,</span> <span class="n">m2_b</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m2</span><span class="p">,</span> <span class="n">m2</span> |
| <span class="n">mean_a</span><span class="p">,</span> <span class="n">mean_b</span> <span class="o">=</span> <span class="n">sum_a</span> <span class="o">/</span> <span class="n">n_a</span><span class="p">,</span> <span class="n">sum_b</span> <span class="o">/</span> <span class="n">n_b</span> |
| <span class="n">delta</span> <span class="o">=</span> <span class="n">mean_b</span> <span class="o">-</span> <span class="n">mean_a</span> |
| <span class="n">combined_n</span> <span class="o">=</span> <span class="n">n_a</span> <span class="o">+</span> <span class="n">n_b</span> |
| <span class="n">m3</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m3</span> <span class="o">+</span> <span class="p">(</span> |
| <span class="p">(</span><span class="n">delta</span><span class="o">**</span><span class="mi">3</span> <span class="o">*</span> <span class="p">((</span><span class="n">n_a</span> <span class="o">*</span> <span class="n">n_b</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">n_a</span> <span class="o">-</span> <span class="n">n_b</span><span class="p">))</span> <span class="o">/</span> <span class="p">((</span><span class="n">combined_n</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span><span class="p">))</span> <span class="o">+</span> |
| <span class="p">((</span><span class="mi">3</span> <span class="o">*</span> <span class="n">delta</span><span class="p">)</span> <span class="o">*</span> <span class="p">((</span><span class="n">n_a</span> <span class="o">*</span> <span class="n">m2_b</span><span class="p">)</span> <span class="o">-</span> <span class="p">(</span><span class="n">n_b</span> <span class="o">*</span> <span class="n">m2_a</span><span class="p">))</span> <span class="o">/</span> <span class="p">(</span><span class="n">combined_n</span><span class="p">)))</span> |
| <span class="n">m2</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m2</span> <span class="o">+</span> <span class="n">delta</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="n">n_b</span> <span class="o">*</span> <span class="n">n_a</span> <span class="o">/</span> <span class="n">combined_n</span> |
| <span class="nb">sum</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">sum</span> |
| <span class="n">n</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> |
| |
| <span class="k">if</span> <span class="n">n</span> <span class="o"><</span> <span class="mi">3</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">float</span><span class="p">(</span><span class="s1">'nan'</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">m2</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">float</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">combined_n</span> <span class="o">*</span> <span class="n">math</span><span class="o">.</span><span class="n">sqrt</span><span class="p">(</span><span class="n">combined_n</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)</span> <span class="o">/</span> <span class="p">(</span><span class="n">combined_n</span> <span class="o">-</span> |
| <span class="mi">2</span><span class="p">)</span> <span class="o">*</span> <span class="n">m3</span> <span class="o">/</span> <span class="p">(</span> |
| <span class="n">m2</span><span class="o">**</span><span class="p">(</span><span class="mi">3</span> <span class="o">/</span> <span class="mi">2</span><span class="p">))</span> |
| |
| <span class="n">moments</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'compute_moments'</span><span class="p">,</span> |
| <span class="n">compute_moments</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'combine_moments'</span><span class="p">,</span> |
| <span class="n">combine_moments</span><span class="p">,</span> <span class="p">[</span><span class="n">moments</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.kurtosis"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.kurtosis">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">kurtosis</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">skipna</span><span class="p">,</span> <span class="n">level</span><span class="p">,</span> <span class="n">numeric_only</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">"per-level aggregation"</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">skipna</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">skipna</span><span class="p">:</span> |
| <span class="bp">self</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">dropna</span><span class="p">()</span> <span class="c1"># pylint: disable=self-cls-assignment</span> |
| |
| <span class="c1"># See the online, numerically stable formulae at</span> |
| <span class="c1"># https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Higher-order_statistics</span> |
| <span class="c1"># kurtosis here calculated as sample kurtosis</span> |
| <span class="c1"># https://en.wikipedia.org/wiki/Kurtosis#Sample_kurtosis</span> |
| <span class="k">def</span> <span class="nf">compute_moments</span><span class="p">(</span><span class="n">x</span><span class="p">):</span> |
| <span class="n">n</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">m2</span><span class="p">,</span> <span class="nb">sum</span><span class="p">,</span> <span class="n">m3</span><span class="p">,</span> <span class="n">m4</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="mi">0</span><span class="p">,</span> <span class="mi">0</span><span class="p">,</span> <span class="mi">0</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">m2</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">std</span><span class="p">(</span><span class="n">ddof</span><span class="o">=</span><span class="mi">0</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="n">n</span> |
| <span class="nb">sum</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">sum</span><span class="p">()</span> |
| <span class="n">m3</span> <span class="o">=</span> <span class="p">(((</span><span class="n">x</span> <span class="o">-</span> <span class="n">x</span><span class="o">.</span><span class="n">mean</span><span class="p">())</span><span class="o">**</span><span class="mi">3</span><span class="p">)</span><span class="o">.</span><span class="n">sum</span><span class="p">())</span> |
| <span class="n">m4</span> <span class="o">=</span> <span class="p">(((</span><span class="n">x</span> <span class="o">-</span> <span class="n">x</span><span class="o">.</span><span class="n">mean</span><span class="p">())</span><span class="o">**</span><span class="mi">4</span><span class="p">)</span><span class="o">.</span><span class="n">sum</span><span class="p">())</span> |
| <span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="nb">dict</span><span class="p">(</span><span class="n">m2</span><span class="o">=</span><span class="p">[</span><span class="n">m2</span><span class="p">],</span> <span class="nb">sum</span><span class="o">=</span><span class="p">[</span><span class="nb">sum</span><span class="p">],</span> <span class="n">n</span><span class="o">=</span><span class="p">[</span><span class="n">n</span><span class="p">],</span> <span class="n">m3</span><span class="o">=</span><span class="p">[</span><span class="n">m3</span><span class="p">],</span> <span class="n">m4</span><span class="o">=</span><span class="p">[</span><span class="n">m4</span><span class="p">]))</span> |
| |
| <span class="k">def</span> <span class="nf">combine_moments</span><span class="p">(</span><span class="n">data</span><span class="p">):</span> |
| <span class="n">m2</span> <span class="o">=</span> <span class="nb">sum</span> <span class="o">=</span> <span class="n">n</span> <span class="o">=</span> <span class="n">m3</span> <span class="o">=</span> <span class="n">m4</span> <span class="o">=</span> <span class="mf">0.0</span> |
| <span class="k">for</span> <span class="n">datum</span> <span class="ow">in</span> <span class="n">data</span><span class="o">.</span><span class="n">itertuples</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">continue</span> |
| <span class="k">elif</span> <span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">m2</span><span class="p">,</span> <span class="nb">sum</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">m3</span><span class="p">,</span> <span class="n">m4</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m2</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">sum</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">m3</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">m4</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">n_a</span><span class="p">,</span> <span class="n">n_b</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span><span class="p">,</span> <span class="n">n</span> |
| <span class="n">m2_a</span><span class="p">,</span> <span class="n">m2_b</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m2</span><span class="p">,</span> <span class="n">m2</span> |
| <span class="n">m3_a</span><span class="p">,</span> <span class="n">m3_b</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m3</span><span class="p">,</span> <span class="n">m3</span> |
| <span class="n">sum_a</span><span class="p">,</span> <span class="n">sum_b</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">sum</span><span class="p">,</span> <span class="nb">sum</span> |
| <span class="n">mean_a</span><span class="p">,</span> <span class="n">mean_b</span> <span class="o">=</span> <span class="n">sum_a</span> <span class="o">/</span> <span class="n">n_a</span><span class="p">,</span> <span class="n">sum_b</span> <span class="o">/</span> <span class="n">n_b</span> |
| <span class="n">delta</span> <span class="o">=</span> <span class="n">mean_b</span> <span class="o">-</span> <span class="n">mean_a</span> |
| <span class="n">combined_n</span> <span class="o">=</span> <span class="n">n_a</span> <span class="o">+</span> <span class="n">n_b</span> |
| <span class="n">m4</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m4</span> <span class="o">+</span> <span class="p">((</span><span class="n">delta</span><span class="o">**</span><span class="mi">4</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">n_a</span> <span class="o">*</span> <span class="n">n_b</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span> |
| <span class="p">(</span><span class="n">n_a</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> <span class="o">-</span> <span class="p">(</span><span class="n">n_a</span> <span class="o">*</span> <span class="n">n_b</span><span class="p">)</span> <span class="o">+</span> |
| <span class="p">(</span><span class="n">n_b</span><span class="o">**</span><span class="mi">2</span><span class="p">))</span> <span class="o">/</span> <span class="n">combined_n</span><span class="o">**</span><span class="mi">3</span><span class="p">)</span> <span class="o">+</span> <span class="p">((</span><span class="mi">6</span> <span class="o">*</span> <span class="n">delta</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> <span class="o">*</span> <span class="p">((</span><span class="n">n_a</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="n">m2_b</span><span class="p">)</span> <span class="o">+</span> |
| <span class="p">(</span><span class="n">n_b</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="n">m2_a</span><span class="p">))</span> <span class="o">/</span> |
| <span class="p">(</span><span class="n">combined_n</span><span class="o">**</span><span class="mi">2</span><span class="p">))</span> <span class="o">+</span> <span class="p">((</span><span class="mi">4</span> <span class="o">*</span> <span class="n">delta</span><span class="p">)</span> <span class="o">*</span> |
| <span class="p">((</span><span class="n">n_a</span> <span class="o">*</span> <span class="n">m3_b</span><span class="p">)</span> <span class="o">-</span> |
| <span class="p">(</span><span class="n">n_b</span> <span class="o">*</span> <span class="n">m3_a</span><span class="p">))</span> <span class="o">/</span> |
| <span class="p">(</span><span class="n">combined_n</span><span class="p">))</span> |
| <span class="n">m3</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m3</span> <span class="o">+</span> <span class="p">(</span> |
| <span class="p">(</span><span class="n">delta</span><span class="o">**</span><span class="mi">3</span> <span class="o">*</span> <span class="p">((</span><span class="n">n_a</span> <span class="o">*</span> <span class="n">n_b</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">n_a</span> <span class="o">-</span> <span class="n">n_b</span><span class="p">))</span> <span class="o">/</span> <span class="p">((</span><span class="n">combined_n</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span><span class="p">))</span> <span class="o">+</span> |
| <span class="p">((</span><span class="mi">3</span> <span class="o">*</span> <span class="n">delta</span><span class="p">)</span> <span class="o">*</span> <span class="p">((</span><span class="n">n_a</span> <span class="o">*</span> <span class="n">m2_b</span><span class="p">)</span> <span class="o">-</span> <span class="p">(</span><span class="n">n_b</span> <span class="o">*</span> <span class="n">m2_a</span><span class="p">))</span> <span class="o">/</span> <span class="p">(</span><span class="n">combined_n</span><span class="p">)))</span> |
| <span class="n">m2</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">m2</span> <span class="o">+</span> <span class="n">delta</span><span class="o">**</span><span class="mi">2</span> <span class="o">*</span> <span class="n">n_b</span> <span class="o">*</span> <span class="n">n_a</span> <span class="o">/</span> <span class="n">combined_n</span> |
| <span class="nb">sum</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">sum</span> |
| <span class="n">n</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> |
| |
| <span class="k">if</span> <span class="n">n</span> <span class="o"><</span> <span class="mi">4</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">float</span><span class="p">(</span><span class="s1">'nan'</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">m2</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">float</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="p">(((</span><span class="n">combined_n</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">combined_n</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">combined_n</span> <span class="o">-</span> <span class="mi">1</span><span class="p">))</span> <span class="o">/</span> |
| <span class="p">((</span><span class="n">combined_n</span> <span class="o">-</span> <span class="mi">2</span><span class="p">)</span> <span class="o">*</span> |
| <span class="p">(</span><span class="n">combined_n</span> <span class="o">-</span> <span class="mi">3</span><span class="p">)))</span> <span class="o">*</span> <span class="p">(</span><span class="n">m4</span> <span class="o">/</span> |
| <span class="p">(</span><span class="n">m2</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> <span class="o">-</span> <span class="p">((</span><span class="mi">3</span> <span class="o">*</span> <span class="p">(</span><span class="n">combined_n</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)</span><span class="o">**</span><span class="mi">2</span><span class="p">)</span> <span class="o">/</span> |
| <span class="p">((</span><span class="n">combined_n</span> <span class="o">-</span> <span class="mi">2</span><span class="p">)</span> <span class="o">*</span> |
| <span class="p">(</span><span class="n">combined_n</span> <span class="o">-</span> <span class="mi">3</span><span class="p">)))</span> |
| |
| <span class="n">moments</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'compute_moments'</span><span class="p">,</span> |
| <span class="n">compute_moments</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'combine_moments'</span><span class="p">,</span> |
| <span class="n">combine_moments</span><span class="p">,</span> <span class="p">[</span><span class="n">moments</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.kurt"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.kurt">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">kurt</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="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="c1"># Compute Kurtosis as kurt is an alias for kurtosis.</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">kurtosis</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></div> |
| |
| <span class="k">def</span> <span class="nf">_corr_aligned</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">):</span> |
| <span class="n">std_x</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">std</span><span class="p">()</span> |
| <span class="n">std_y</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">std</span><span class="p">()</span> |
| <span class="n">cov</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cov_aligned</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">cov</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">cov</span><span class="p">,</span> <span class="n">std_x</span><span class="p">,</span> <span class="n">std_y</span><span class="p">:</span> <span class="n">cov</span> <span class="o">/</span> <span class="p">(</span><span class="n">std_x</span> <span class="o">*</span> <span class="n">std_y</span><span class="p">),</span> <span class="n">args</span><span class="o">=</span><span class="p">[</span><span class="n">std_x</span><span class="p">,</span> <span class="n">std_y</span><span class="p">])</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.cov"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.cov">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">cov</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">,</span> <span class="n">ddof</span><span class="p">):</span> |
| <span class="n">x</span><span class="p">,</span> <span class="n">y</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">dropna</span><span class="p">()</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">dropna</span><span class="p">(),</span> <span class="s1">'inner'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">x</span><span class="o">.</span><span class="n">_cov_aligned</span><span class="p">(</span><span class="n">y</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">,</span> <span class="n">ddof</span><span class="p">)</span></div> |
| |
| <span class="k">def</span> <span class="nf">_cov_aligned</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">,</span> <span class="n">ddof</span><span class="o">=</span><span class="mi">1</span><span class="p">):</span> |
| <span class="c1"># Use the formulae from</span> |
| <span class="c1"># https://en.wikipedia.org/wiki/Algorithms_for_calculating_variance#Covariance</span> |
| <span class="k">def</span> <span class="nf">compute_co_moments</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">):</span> |
| <span class="n">n</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">n</span> <span class="o"><=</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="n">c</span> <span class="o">=</span> <span class="mi">0</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">c</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">cov</span><span class="p">(</span><span class="n">y</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">n</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)</span> |
| <span class="n">sx</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">sum</span><span class="p">()</span> |
| <span class="n">sy</span> <span class="o">=</span> <span class="n">y</span><span class="o">.</span><span class="n">sum</span><span class="p">()</span> |
| <span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="nb">dict</span><span class="p">(</span><span class="n">c</span><span class="o">=</span><span class="p">[</span><span class="n">c</span><span class="p">],</span> <span class="n">sx</span><span class="o">=</span><span class="p">[</span><span class="n">sx</span><span class="p">],</span> <span class="n">sy</span><span class="o">=</span><span class="p">[</span><span class="n">sy</span><span class="p">],</span> <span class="n">n</span><span class="o">=</span><span class="p">[</span><span class="n">n</span><span class="p">]))</span> |
| |
| <span class="k">def</span> <span class="nf">combine_co_moments</span><span class="p">(</span><span class="n">data</span><span class="p">):</span> |
| <span class="n">c</span> <span class="o">=</span> <span class="n">sx</span> <span class="o">=</span> <span class="n">sy</span> <span class="o">=</span> <span class="n">n</span> <span class="o">=</span> <span class="mf">0.0</span> |
| <span class="k">for</span> <span class="n">datum</span> <span class="ow">in</span> <span class="n">data</span><span class="o">.</span><span class="n">itertuples</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="k">continue</span> |
| <span class="k">elif</span> <span class="n">n</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">c</span><span class="p">,</span> <span class="n">sx</span><span class="p">,</span> <span class="n">sy</span><span class="p">,</span> <span class="n">n</span> <span class="o">=</span> <span class="n">datum</span><span class="o">.</span><span class="n">c</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">sx</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">sy</span><span class="p">,</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">c</span> <span class="o">+=</span> <span class="p">(</span> |
| <span class="n">datum</span><span class="o">.</span><span class="n">c</span> <span class="o">+</span> <span class="p">(</span><span class="n">sx</span> <span class="o">/</span> <span class="n">n</span> <span class="o">-</span> <span class="n">datum</span><span class="o">.</span><span class="n">sx</span> <span class="o">/</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span><span class="p">)</span> <span class="o">*</span> |
| <span class="p">(</span><span class="n">sy</span> <span class="o">/</span> <span class="n">n</span> <span class="o">-</span> <span class="n">datum</span><span class="o">.</span><span class="n">sy</span> <span class="o">/</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span><span class="p">)</span> <span class="o">*</span> <span class="n">n</span> <span class="o">*</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> <span class="o">/</span> <span class="p">(</span><span class="n">n</span> <span class="o">+</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span><span class="p">))</span> |
| <span class="n">sx</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">sx</span> |
| <span class="n">sy</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">sy</span> |
| <span class="n">n</span> <span class="o">+=</span> <span class="n">datum</span><span class="o">.</span><span class="n">n</span> |
| <span class="k">if</span> <span class="n">n</span> <span class="o"><</span> <span class="nb">max</span><span class="p">(</span><span class="mi">2</span><span class="p">,</span> <span class="n">ddof</span><span class="p">,</span> <span class="n">min_periods</span> <span class="ow">or</span> <span class="mi">0</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">float</span><span class="p">(</span><span class="s1">'nan'</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">c</span> <span class="o">/</span> <span class="p">(</span><span class="n">n</span> <span class="o">-</span> <span class="n">ddof</span><span class="p">)</span> |
| |
| <span class="n">moments</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'compute_co_moments'</span><span class="p">,</span> |
| <span class="n">compute_co_moments</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'combine_co_moments'</span><span class="p">,</span> |
| <span class="n">combine_co_moments</span><span class="p">,</span> <span class="p">[</span><span class="n">moments</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.dropna"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.dropna">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">dropna</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'dropna'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">dropna</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.set_axis"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.set_axis">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">set_axis</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">labels</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="c1"># TODO: assigning the index is generally order-sensitive, but we could</span> |
| <span class="c1"># support it in some rare cases, e.g. when assigning the index from one</span> |
| <span class="c1"># of a DataFrame's columns</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="s2">"Assigning an index is not yet supported. "</span> |
| <span class="s2">"Consider using set_index() instead."</span><span class="p">)</span></div> |
| |
| <span class="n">isnull</span> <span class="o">=</span> <span class="n">isna</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'isna'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| <span class="n">notnull</span> <span class="o">=</span> <span class="n">notna</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'notna'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| |
| <span class="n">items</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'items'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">iteritems</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'iteritems'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">tolist</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'tolist'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">to_numpy</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_numpy'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">to_string</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_string'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_wrap_in_df</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'wrap_in_df'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</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">s</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="p">))</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.duplicated"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.duplicated">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">duplicated</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keep</span><span class="p">):</span> |
| <span class="sd">"""Only ``keep=False`` and ``keep="any"`` are supported. Other values of</span> |
| <span class="sd"> ``keep`` make this an order-sensitive operation. Note ``keep="any"`` is</span> |
| <span class="sd"> a Beam-specific option that guarantees only one duplicate will be kept, but</span> |
| <span class="sd"> unlike ``"first"`` and ``"last"`` it makes no guarantees about _which_</span> |
| <span class="sd"> duplicate element is kept."""</span> |
| <span class="c1"># Re-use the DataFrame based duplcated, extract the series back out</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_wrap_in_df</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">duplicated</span><span class="p">(</span><span class="n">keep</span><span class="o">=</span><span class="n">keep</span><span class="p">)[</span><span class="n">df</span><span class="o">.</span><span class="n">columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]]</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.drop_duplicates"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.drop_duplicates">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">drop_duplicates</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keep</span><span class="p">):</span> |
| <span class="sd">"""Only ``keep=False`` and ``keep="any"`` are supported. Other values of</span> |
| <span class="sd"> ``keep`` make this an order-sensitive operation. Note ``keep="any"`` is</span> |
| <span class="sd"> a Beam-specific option that guarantees only one duplicate will be kept, but</span> |
| <span class="sd"> unlike ``"first"`` and ``"last"`` it makes no guarantees about _which_</span> |
| <span class="sd"> duplicate element is kept."""</span> |
| <span class="c1"># Re-use the DataFrame based drop_duplicates, extract the series back out</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_wrap_in_df</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">drop_duplicates</span><span class="p">(</span><span class="n">keep</span><span class="o">=</span><span class="n">keep</span><span class="p">)[</span><span class="n">df</span><span class="o">.</span><span class="n">columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]]</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.sample"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.sample">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">sample</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Only ``n`` and/or ``weights`` may be specified. ``frac``,</span> |
| <span class="sd"> ``random_state``, and ``replace=True`` are not yet supported.</span> |
| <span class="sd"> See `BEAM-12476 <https://issues.apache.org/jira/BEAM-12476>`_.</span> |
| |
| <span class="sd"> Note that pandas will raise an error if ``n`` is larger than the length</span> |
| <span class="sd"> of the dataset, while the Beam DataFrame API will simply return the full</span> |
| <span class="sd"> dataset in that case."""</span> |
| |
| <span class="c1"># Re-use the DataFrame based sample, extract the series back out</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_wrap_in_df</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">sample</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)[</span><span class="n">df</span><span class="o">.</span><span class="n">columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]]</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.aggregate"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.aggregate">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">aggregate</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">axis</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="sd">"""Some aggregation methods cannot be parallelized, and computing</span> |
| <span class="sd"> them will require collecting all data on a single machine."""</span> |
| <span class="k">if</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'skipna'</span><span class="p">,</span> <span class="kc">False</span><span class="p">):</span> |
| <span class="c1"># Eagerly generate a proxy to make sure skipna is a valid argument</span> |
| <span class="c1"># for this aggregation method</span> |
| <span class="n">_</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">aggregate</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">axis</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="n">kwargs</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="s1">'skipna'</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">dropna</span><span class="p">()</span><span class="o">.</span><span class="n">aggregate</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">func</span><span class="p">)</span> <span class="o">></span> <span class="mi">1</span><span class="p">:</span> |
| <span class="c1"># level arg is ignored for multiple aggregations</span> |
| <span class="n">_</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="s1">'level'</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> |
| |
| <span class="c1"># Aggregate with each method separately, then stick them all together.</span> |
| <span class="n">rows</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">agg</span><span class="p">([</span><span class="n">f</span><span class="p">],</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="k">for</span> <span class="n">f</span> <span class="ow">in</span> <span class="n">func</span><span class="p">]</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'join_aggregate'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="o">*</span><span class="n">rows</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">concat</span><span class="p">(</span><span class="n">rows</span><span class="p">),</span> <span class="p">[</span><span class="n">row</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">row</span> <span class="ow">in</span> <span class="n">rows</span><span class="p">]))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># We're only handling a single column. It could be 'func' or ['func'],</span> |
| <span class="c1"># which produce different results. 'func' produces a scalar, ['func']</span> |
| <span class="c1"># produces a single element Series.</span> |
| <span class="n">base_func</span> <span class="o">=</span> <span class="n">func</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="k">else</span> <span class="n">func</span> |
| |
| <span class="k">if</span> <span class="p">(</span><span class="n">_is_numeric</span><span class="p">(</span><span class="n">base_func</span><span class="p">)</span> <span class="ow">and</span> |
| <span class="ow">not</span> <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">dtypes</span><span class="o">.</span><span class="n">common</span><span class="o">.</span><span class="n">is_numeric_dtype</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">dtype</span><span class="p">)):</span> |
| <span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"Performing a numeric aggregation, </span><span class="si">{</span><span class="n">base_func</span><span class="si">!r}</span><span class="s2">, on "</span> |
| <span class="sa">f</span><span class="s2">"Series </span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">name</span><span class="si">!r}</span><span class="s2"> with non-numeric type "</span> |
| <span class="sa">f</span><span class="s2">"</span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">dtype</span><span class="si">!r}</span><span class="s2">. This can result in runtime errors or surprising "</span> |
| <span class="s2">"results."</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="s1">'level'</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="p">:</span> |
| <span class="c1"># Defer to groupby.agg for level= mode</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span> |
| <span class="n">level</span><span class="o">=</span><span class="n">kwargs</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="s1">'level'</span><span class="p">),</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">)</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</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="n">singleton_reason</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">if</span> <span class="s1">'min_count'</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="p">:</span> |
| <span class="c1"># Eagerly generate a proxy to make sure min_count is a valid argument</span> |
| <span class="c1"># for this aggregation method</span> |
| <span class="n">_</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">axis</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="n">singleton_reason</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s2">"Aggregation with min_count= requires collecting all data on a "</span> |
| <span class="s2">"single node."</span><span class="p">)</span> |
| |
| <span class="c1"># We have specialized distributed implementations for these</span> |
| <span class="k">if</span> <span class="n">base_func</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'quantile'</span><span class="p">,</span> |
| <span class="s1">'std'</span><span class="p">,</span> |
| <span class="s1">'var'</span><span class="p">,</span> |
| <span class="s1">'nunique'</span><span class="p">,</span> |
| <span class="s1">'corr'</span><span class="p">,</span> |
| <span class="s1">'cov'</span><span class="p">,</span> |
| <span class="s1">'skew'</span><span class="p">,</span> |
| <span class="s1">'kurt'</span><span class="p">,</span> |
| <span class="s1">'kurtosis'</span><span class="p">):</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">base_func</span><span class="p">)(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s1">'wrap_aggregate_</span><span class="si">{</span><span class="n">base_func</span><span class="si">}</span><span class="s1">'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">x</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">x</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="p">[</span><span class="n">base_func</span><span class="p">]),</span> <span class="p">[</span><span class="n">result</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">result</span> |
| |
| <span class="n">agg_kwargs</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="k">if</span> <span class="p">((</span><span class="n">_is_associative</span><span class="p">(</span><span class="n">base_func</span><span class="p">)</span> <span class="ow">or</span> <span class="n">_is_liftable_with_sum</span><span class="p">(</span><span class="n">base_func</span><span class="p">))</span> <span class="ow">and</span> |
| <span class="n">singleton_reason</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">):</span> |
| <span class="n">intermediate</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s1">'pre_aggregate_</span><span class="si">{</span><span class="n">base_func</span><span class="si">}</span><span class="s1">'</span><span class="p">,</span> |
| <span class="c1"># Coerce to a Series, if the result is scalar we still want a Series</span> |
| <span class="c1"># so we can combine and do the final aggregation next.</span> |
| <span class="k">lambda</span> <span class="n">s</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">s</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| <span class="n">allow_nonparallel_final</span> <span class="o">=</span> <span class="kc">True</span> |
| <span class="k">if</span> <span class="n">_is_associative</span><span class="p">(</span><span class="n">base_func</span><span class="p">):</span> |
| <span class="n">agg_func</span> <span class="o">=</span> <span class="n">func</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">agg_func</span> <span class="o">=</span> <span class="p">[</span><span class="s1">'sum'</span><span class="p">]</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="k">else</span> <span class="s1">'sum'</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">intermediate</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> |
| <span class="n">allow_nonparallel_final</span> <span class="o">=</span> <span class="kc">None</span> <span class="c1"># i.e. don't change the value</span> |
| <span class="n">agg_func</span> <span class="o">=</span> <span class="n">func</span> |
| <span class="n">singleton_reason</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"Aggregation function </span><span class="si">{</span><span class="n">func</span><span class="si">!r}</span><span class="s2"> cannot currently be "</span> |
| <span class="s2">"parallelized. It requires collecting all data for "</span> |
| <span class="s2">"this Series on a single node."</span><span class="p">)</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="n">allow_nonparallel_final</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s1">'post_aggregate_</span><span class="si">{</span><span class="n">base_func</span><span class="si">}</span><span class="s1">'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">agg_func</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">agg_kwargs</span><span class="p">),</span> <span class="p">[</span><span class="n">intermediate</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="n">singleton_reason</span><span class="p">)))</span></div> |
| |
| <span class="n">agg</span> <span class="o">=</span> <span class="n">aggregate</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">axes</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="bp">self</span><span class="o">.</span><span class="n">index</span><span class="p">]</span> |
| |
| <span class="n">clip</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'clip'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| |
| <span class="nb">all</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'all'</span><span class="p">)</span> |
| <span class="nb">any</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'any'</span><span class="p">)</span> |
| <span class="c1"># TODO(BEAM-12074): Document that Series.count(level=) will drop NaN's</span> |
| <span class="n">count</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'count'</span><span class="p">)</span> |
| <span class="n">describe</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'describe'</span><span class="p">)</span> |
| <span class="nb">min</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'min'</span><span class="p">)</span> |
| <span class="nb">max</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'max'</span><span class="p">)</span> |
| <span class="n">prod</span> <span class="o">=</span> <span class="n">product</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'prod'</span><span class="p">)</span> |
| <span class="nb">sum</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'sum'</span><span class="p">)</span> |
| <span class="n">mean</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'mean'</span><span class="p">)</span> |
| <span class="n">median</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'median'</span><span class="p">)</span> |
| <span class="n">sem</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'sem'</span><span class="p">)</span> |
| <span class="n">mad</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'mad'</span><span class="p">)</span> |
| |
| <span class="n">argmax</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'argmax'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">argmin</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'argmin'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cummax</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'cummax'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cummin</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'cummin'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cumprod</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'cumprod'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cumsum</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'cumsum'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">diff</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'diff'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">interpolate</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'interpolate'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">searchsorted</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'searchsorted'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">shift</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'shift'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">pct_change</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'pct_change'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">is_monotonic</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'is_monotonic'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">is_monotonic_increasing</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'is_monotonic_increasing'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">is_monotonic_decreasing</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'is_monotonic_decreasing'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">asof</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'asof'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">first_valid_index</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'first_valid_index'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">last_valid_index</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'last_valid_index'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">autocorr</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'autocorr'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">iat</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'iat'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">))</span> |
| |
| <span class="n">head</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'head'</span><span class="p">,</span> <span class="n">explanation</span><span class="o">=</span><span class="n">_PEEK_METHOD_EXPLANATION</span><span class="p">)</span> |
| <span class="n">tail</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'tail'</span><span class="p">,</span> <span class="n">explanation</span><span class="o">=</span><span class="n">_PEEK_METHOD_EXPLANATION</span><span class="p">)</span> |
| |
| <span class="nb">filter</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'filter'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| |
| <span class="n">memory_usage</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'memory_usage'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">nbytes</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'nbytes'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">to_list</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_list'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="n">factorize</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'factorize'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| |
| <span class="c1"># In Series __contains__ checks the index</span> |
| <span class="fm">__contains__</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'__contains__'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.nlargest"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.nlargest">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">nlargest</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keep</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Only ``keep=False`` and ``keep="any"`` are supported. Other values of</span> |
| <span class="sd"> ``keep`` make this an order-sensitive operation. Note ``keep="any"`` is</span> |
| <span class="sd"> a Beam-specific option that guarantees only one duplicate will be kept, but</span> |
| <span class="sd"> unlike ``"first"`` and ``"last"`` it makes no guarantees about _which_</span> |
| <span class="sd"> duplicate element is kept."""</span> |
| <span class="c1"># TODO(robertwb): Document 'any' option.</span> |
| <span class="c1"># TODO(robertwb): Consider (conditionally) defaulting to 'any' if no</span> |
| <span class="c1"># explicit keep parameter is requested.</span> |
| <span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s1">'any'</span><span class="p">:</span> |
| <span class="n">keep</span> <span class="o">=</span> <span class="s1">'first'</span> |
| <span class="k">elif</span> <span class="n">keep</span> <span class="o">!=</span> <span class="s1">'all'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"nlargest(keep=</span><span class="si">{</span><span class="n">keep</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"order sensitive. Only keep=</span><span class="se">\"</span><span class="s2">all</span><span class="se">\"</span><span class="s2"> is supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="s1">'keep'</span><span class="p">]</span> <span class="o">=</span> <span class="n">keep</span> |
| <span class="n">per_partition</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'nlargest-per-partition'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">nlargest</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'nlargest'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">nlargest</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="n">per_partition</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.nsmallest"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.nsmallest">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">nsmallest</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keep</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Only ``keep=False`` and ``keep="any"`` are supported. Other values of</span> |
| <span class="sd"> ``keep`` make this an order-sensitive operation. Note ``keep="any"`` is</span> |
| <span class="sd"> a Beam-specific option that guarantees only one duplicate will be kept, but</span> |
| <span class="sd"> unlike ``"first"`` and ``"last"`` it makes no guarantees about _which_</span> |
| <span class="sd"> duplicate element is kept."""</span> |
| <span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s1">'any'</span><span class="p">:</span> |
| <span class="n">keep</span> <span class="o">=</span> <span class="s1">'first'</span> |
| <span class="k">elif</span> <span class="n">keep</span> <span class="o">!=</span> <span class="s1">'all'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"nsmallest(keep=</span><span class="si">{</span><span class="n">keep</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"order sensitive. Only keep=</span><span class="se">\"</span><span class="s2">all</span><span class="se">\"</span><span class="s2"> is supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="s1">'keep'</span><span class="p">]</span> <span class="o">=</span> <span class="n">keep</span> |
| <span class="n">per_partition</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'nsmallest-per-partition'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">nsmallest</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'nsmallest'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">nsmallest</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="n">per_partition</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">is_unique</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">set_index</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="n">s</span> <span class="o">=</span> <span class="n">s</span><span class="p">[:]</span> |
| <span class="n">s</span><span class="o">.</span><span class="n">index</span> <span class="o">=</span> <span class="n">s</span> |
| <span class="k">return</span> <span class="n">s</span> |
| |
| <span class="n">self_index</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'set_index'</span><span class="p">,</span> |
| <span class="n">set_index</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="n">is_unique_distributed</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'is_unique_distributed'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</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">s</span><span class="o">.</span><span class="n">is_unique</span><span class="p">),</span> <span class="p">[</span><span class="n">self_index</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">():</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'combine'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">all</span><span class="p">(),</span> <span class="p">[</span><span class="n">is_unique_distributed</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| <span class="n">plot</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'plot'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| <span class="n">pop</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'pop'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="n">rename_axis</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'rename_axis'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| |
| <span class="nb">round</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'round'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| |
| <span class="n">take</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'take'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'deprecated'</span><span class="p">)</span> |
| |
| <span class="n">to_dict</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_dict'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="n">to_frame</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'to_frame'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.unique"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.unique">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">unique</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">as_series</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="sd">"""unique is not supported by default because it produces a</span> |
| <span class="sd"> non-deferred result: an :class:`~numpy.ndarray`. You can use the</span> |
| <span class="sd"> Beam-specific argument ``unique(as_series=True)`` to get the result as</span> |
| <span class="sd"> a :class:`DeferredSeries`"""</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">as_series</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"unique() is not supported by default because it produces a "</span> |
| <span class="s2">"non-deferred result: a numpy array. You can use the Beam-specific "</span> |
| <span class="s2">"argument unique(as_series=True) to get the result as a "</span> |
| <span class="s2">"DeferredSeries"</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'unique'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</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">df</span><span class="o">.</span><span class="n">unique</span><span class="p">()),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"unique() cannot currently be parallelized."</span><span class="p">)))</span></div> |
| |
| <div class="viewcode-block" id="DeferredSeries.update"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.update">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">update</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'update'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> |
| <span class="n">other</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">other</span><span class="p">)</span> <span class="ow">or</span> <span class="n">df</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">())</span></div> |
| |
| <span class="n">unstack</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'unstack'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'non-deferred-columns'</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.value_counts"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.value_counts">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">value_counts</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">sort</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">normalize</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">ascending</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">bins</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> |
| <span class="n">dropna</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="sd">"""``sort`` is ``False`` by default, and ``sort=True`` is not supported</span> |
| <span class="sd"> because it imposes an ordering on the dataset which likely will not be</span> |
| <span class="sd"> preserved.</span> |
| |
| <span class="sd"> When ``bin`` is specified this operation is not parallelizable. See</span> |
| <span class="sd"> [BEAM-12441](https://issues.apache.org/jira/browse/BEAM-12441) tracking the</span> |
| <span class="sd"> possible addition of a distributed implementation."""</span> |
| |
| <span class="k">if</span> <span class="n">sort</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"value_counts(sort=True) is not supported because it imposes an "</span> |
| <span class="s2">"ordering on the dataset which likely will not be preserved."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">bins</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">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'value_counts'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">value_counts</span><span class="p">(</span> |
| <span class="n">normalize</span><span class="o">=</span><span class="n">normalize</span><span class="p">,</span> <span class="n">bins</span><span class="o">=</span><span class="n">bins</span><span class="p">,</span> <span class="n">dropna</span><span class="o">=</span><span class="n">dropna</span><span class="p">)[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"value_counts with bin specified requires collecting "</span> |
| <span class="s2">"the entire dataset to identify the range."</span><span class="p">)),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="p">))</span> |
| |
| <span class="k">if</span> <span class="n">dropna</span><span class="p">:</span> |
| <span class="n">column</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">dropna</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">column</span> <span class="o">=</span> <span class="bp">self</span> |
| |
| <span class="n">result</span> <span class="o">=</span> <span class="n">column</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">column</span><span class="p">,</span> <span class="n">dropna</span><span class="o">=</span><span class="n">dropna</span><span class="p">)</span><span class="o">.</span><span class="n">size</span><span class="p">()</span> |
| |
| <span class="c1"># groupby.size() names the index, which we don't need</span> |
| <span class="n">result</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="k">if</span> <span class="n">normalize</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">result</span> <span class="o">/</span> <span class="n">column</span><span class="o">.</span><span class="n">length</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">result</span></div> |
| |
| <span class="n">values</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'values'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">))</span> |
| |
| <span class="n">view</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'view'</span><span class="p">,</span> |
| <span class="n">explanation</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"because it relies on memory-sharing semantics that are "</span> |
| <span class="s2">"not compatible with the Beam model."</span><span class="p">))</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">str</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_DeferredStringMethods</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">cat</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_DeferredCategoricalMethods</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">dt</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_DeferredDatetimeMethods</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.mode"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.mode">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">mode</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="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""mode is not currently parallelizable. An approximate,</span> |
| <span class="sd"> parallelizable implementation of mode may be added in the future</span> |
| <span class="sd"> (`BEAM-12181 <https://issues.apache.org/jira/BEAM-12181>`_)."""</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'mode'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">mode</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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="c1">#TODO(BEAM-12181): Can we add an approximate implementation?</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span> |
| <span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"mode cannot currently be parallelized. See "</span> |
| <span class="s2">"BEAM-12181 tracking the possble addition of "</span> |
| <span class="s2">"an approximate, parallelizable implementation of mode."</span><span class="p">)),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <span class="n">apply</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'apply'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| <span class="nb">map</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'map'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> |
| <span class="c1"># TODO(BEAM-11636): Implement transform using type inference to determine the</span> |
| <span class="c1"># proxy</span> |
| <span class="c1">#transform = frame_base._elementwise_method('transform', base=pd.Series)</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.repeat"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.repeat">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">repeat</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">repeats</span><span class="p">,</span> <span class="n">axis</span><span class="p">):</span> |
| <span class="sd">"""``repeats`` must be an ``int`` or a :class:`DeferredSeries`. Lists are</span> |
| <span class="sd"> not supported because they make this operation order-sensitive."""</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">repeats</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'repeat'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">series</span><span class="p">:</span> <span class="n">series</span><span class="o">.</span><span class="n">repeat</span><span class="p">(</span><span class="n">repeats</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">repeats</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'repeat'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">series</span><span class="p">,</span> |
| <span class="n">repeats_series</span><span class="p">:</span> <span class="n">series</span><span class="o">.</span><span class="n">repeat</span><span class="p">(</span><span class="n">repeats_series</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">repeats</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">repeats</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"repeat(repeats=) repeats must be an int or a DeferredSeries. "</span> |
| <span class="s2">"Lists are not supported because they make this operation sensitive "</span> |
| <span class="s2">"to the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</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">"repeat(repeats=) value must be an int or a "</span> |
| <span class="sa">f</span><span class="s2">"DeferredSeries (encountered </span><span class="si">{</span><span class="nb">type</span><span class="p">(</span><span class="n">repeats</span><span class="p">)</span><span class="si">}</span><span class="s2">)."</span><span class="p">)</span></div> |
| |
| <span class="k">if</span> <span class="nb">hasattr</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="s1">'compare'</span><span class="p">):</span> |
| |
| <div class="viewcode-block" id="DeferredSeries.compare"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredSeries.compare">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">compare</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">align_axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| |
| <span class="k">if</span> <span class="n">align_axis</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'index'</span><span class="p">,</span> <span class="mi">0</span><span class="p">):</span> |
| <span class="n">preserves_partition</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()</span> |
| <span class="k">elif</span> <span class="n">align_axis</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'columns'</span><span class="p">,</span> <span class="mi">1</span><span class="p">):</span> |
| <span class="n">preserves_partition</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"align_axis must be one of ('index', 0, 'columns', 1). "</span> |
| <span class="sa">f</span><span class="s2">"got </span><span class="si">{</span><span class="n">align_axis</span><span class="si">!r}</span><span class="s2">."</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'compare'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">,</span> |
| <span class="n">other</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">compare</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">align_axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">preserves_partition</span><span class="p">))</span></div></div> |
| |
| |
| <div class="viewcode-block" id="DeferredDataFrame"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame">[docs]</a><span class="nd">@populate_not_implemented</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="nd">@frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">_register_for</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">DeferredDataFrame</span><span class="p">(</span><span class="n">DeferredDataFrameOrSeries</span><span class="p">):</span> |
| <span class="k">def</span> <span class="fm">__repr__</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="sa">f</span><span class="s1">'DeferredDataFrame(columns=</span><span class="si">{</span><span class="nb">list</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="si">}</span><span class="s1">, '</span> |
| <span class="sa">f</span><span class="s1">'</span><span class="si">{</span><span class="bp">self</span><span class="o">.</span><span class="n">_render_indexes</span><span class="p">()</span><span class="si">}</span><span class="s1">)'</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">columns</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span> |
| |
| <span class="nd">@columns</span><span class="o">.</span><span class="n">setter</span> |
| <span class="k">def</span> <span class="nf">columns</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">columns</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">set_columns</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">columns</span> <span class="o">=</span> <span class="n">columns</span> |
| <span class="k">return</span> <span class="n">df</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'set_columns'</span><span class="p">,</span> |
| <span class="n">set_columns</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.keys"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.keys">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">keys</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span></div> |
| |
| <span class="k">def</span> <span class="fm">__getattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">):</span> |
| <span class="c1"># Column attribute access.</span> |
| <span class="k">if</span> <span class="n">name</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">name</span><span class="p">]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">object</span><span class="o">.</span><span class="fm">__getattribute__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="fm">__getitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">):</span> |
| <span class="c1"># TODO: Replicate pd.DataFrame.__getitem__ logic</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">)</span> <span class="ow">and</span> <span class="n">key</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dtype</span> <span class="o">==</span> <span class="nb">bool</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> |
| |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="c1"># Fail early if key is a DeferredBase as it interacts surprisingly with</span> |
| <span class="c1"># key in self._expr.proxy().columns</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="s2">"Indexing with a non-bool deferred frame is not yet supported. "</span> |
| <span class="s2">"Consider using df.loc[...]"</span><span class="p">)</span> |
| |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">slice</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">_is_null_slice</span><span class="p">(</span><span class="n">key</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span> |
| <span class="k">elif</span> <span class="n">_is_integer_slice</span><span class="p">(</span><span class="n">key</span><span class="p">):</span> |
| <span class="c1"># This depends on the contents of the index.</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"Integer slices are not supported as they are ambiguous. Please "</span> |
| <span class="s2">"use iloc or loc with integer slices."</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> |
| |
| <span class="k">elif</span> <span class="p">(</span> |
| <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">all</span><span class="p">(</span><span class="n">key_column</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span> |
| <span class="k">for</span> <span class="n">key_column</span> <span class="ow">in</span> <span class="n">key</span><span class="p">))</span> <span class="ow">or</span> |
| <span class="n">key</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_elementwise</span><span class="p">(</span><span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="p">[</span><span class="n">key</span><span class="p">],</span> <span class="s1">'get_column'</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="n">key</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="fm">__contains__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">):</span> |
| <span class="c1"># Checks if proxy has the given column</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="fm">__contains__</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="fm">__setitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span> |
| <span class="n">key</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="ow">or</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> |
| <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">c</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">key</span><span class="p">))</span> <span class="ow">or</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">)</span> <span class="ow">and</span> |
| <span class="n">key</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dtype</span> <span class="o">==</span> <span class="nb">bool</span><span class="p">):</span> |
| <span class="c1"># yapf: disable</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_elementwise</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="fm">__setitem__</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">),</span> |
| <span class="s1">'set_column'</span><span class="p">,</span> |
| <span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">),</span> |
| <span class="n">inplace</span><span class="o">=</span><span class="kc">True</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="n">key</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.align"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.align">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">align</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">join</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">copy</span><span class="p">,</span> <span class="n">level</span><span class="p">,</span> <span class="n">method</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Aligning per level is not yet supported. Only the default,</span> |
| <span class="sd"> ``level=None``, is allowed.</span> |
| |
| <span class="sd"> Filling NaN values via ``method`` is not supported, because it is</span> |
| <span class="sd"> `order-sensitive</span> |
| <span class="sd"> <https://s.apache.org/dataframe-order-sensitive-operations>`_. Only the</span> |
| <span class="sd"> default, ``method=None``, is allowed.</span> |
| |
| <span class="sd"> ``copy=False`` is not supported because its behavior (whether or not it is</span> |
| <span class="sd"> an inplace operation) depends on the data."""</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">copy</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"align(copy=False) is not supported because it might be an inplace "</span> |
| <span class="s2">"operation depending on the data. Please prefer the default "</span> |
| <span class="s2">"align(copy=True)."</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">method</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"align(method=</span><span class="si">{</span><span class="n">method</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"order sensitive. Only align(method=None) is supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">kwargs</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">'align(</span><span class="si">%s</span><span class="s1">)'</span> <span class="o">%</span> <span class="s1">', '</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">kwargs</span><span class="o">.</span><span class="n">keys</span><span class="p">()))</span> |
| |
| <span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># Could probably get by partitioning on the used levels.</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"align(level=</span><span class="si">{</span><span class="n">level</span><span class="si">}</span><span class="s2">) is not currently parallelizable. Only "</span> |
| <span class="s2">"align(level=None) can be parallelized."</span><span class="p">))</span> |
| <span class="k">elif</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'columns'</span><span class="p">,</span> <span class="mi">1</span><span class="p">):</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'align'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">join</span><span class="o">=</span><span class="n">join</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires_partition_by</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.append"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.append">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">append</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">ignore_index</span><span class="p">,</span> <span class="n">verify_integrity</span><span class="p">,</span> <span class="n">sort</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``ignore_index=True`` is not supported, because it requires generating an</span> |
| <span class="sd"> order-sensitive index."""</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DeferredDataFrame</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"append() only accepts DeferredDataFrame instances, received "</span> <span class="o">+</span> |
| <span class="nb">str</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">)))</span> |
| <span class="k">if</span> <span class="n">ignore_index</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"append(ignore_index=True) is order sensitive because it requires "</span> |
| <span class="s2">"generating a new index based on the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">verify_integrity</span><span class="p">:</span> |
| <span class="c1"># We can verify the index is non-unique within index partitioned data.</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'append'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">sort</span><span class="o">=</span><span class="n">sort</span><span class="p">,</span> |
| <span class="n">verify_integrity</span><span class="o">=</span><span class="n">verify_integrity</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="p">)</span> |
| <span class="p">)</span></div> |
| |
| <span class="c1"># If column name exists this is a simple project, otherwise it is a constant</span> |
| <span class="c1"># (default_value)</span> |
| <div class="viewcode-block" id="DeferredDataFrame.get"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.get">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">get</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">default_value</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">key</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">:</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">default_value</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.set_index"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.set_index">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">set_index</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keys</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``keys`` must be a ``str`` or ``List[str]``. Passing an Index or Series</span> |
| <span class="sd"> is not yet supported (`BEAM-11711</span> |
| <span class="sd"> <https://issues.apache.org/jira/browse/BEAM-11711>`_)."""</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">keys</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span> |
| <span class="n">keys</span> <span class="o">=</span> <span class="p">[</span><span class="n">keys</span><span class="p">]</span> |
| |
| <span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="p">(</span><span class="n">_DeferredIndex</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">))</span> |
| <span class="k">for</span> <span class="n">k</span> <span class="ow">in</span> <span class="n">keys</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">"set_index with Index or Series instances is "</span> |
| <span class="s2">"not yet supported (BEAM-11711)."</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'set_index'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">keys</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| |
| <div class="viewcode-block" id="DeferredDataFrame.set_axis"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.set_axis">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">set_axis</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">labels</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'index'</span><span class="p">,</span> <span class="mi">0</span><span class="p">):</span> |
| <span class="c1"># TODO: assigning the index is generally order-sensitive, but we could</span> |
| <span class="c1"># support it in some rare cases, e.g. when assigning the index from one</span> |
| <span class="c1"># of a DataFrame's columns</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="s2">"Assigning an index is not yet supported. "</span> |
| <span class="s2">"Consider using set_index() instead."</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'set_axis'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">set_axis</span><span class="p">(</span><span class="n">labels</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span></div> |
| |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">axes</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="bp">self</span><span class="o">.</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="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">dtypes</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dtypes</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.assign"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.assign">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">assign</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``value`` must be a ``callable`` or :class:`DeferredSeries`. Other types</span> |
| <span class="sd"> make this operation order-sensitive."""</span> |
| <span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">callable</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"Unsupported value for new column '</span><span class="si">{</span><span class="n">name</span><span class="si">}</span><span class="s2">': '</span><span class="si">{</span><span class="n">value</span><span class="si">}</span><span class="s2">'. Only "</span> |
| <span class="s2">"callables and DeferredSeries instances are supported. Other types "</span> |
| <span class="s2">"make this operation sensitive to the order of the data"</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_elementwise</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">df</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="n">df</span><span class="o">.</span><span class="n">assign</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="s1">'assign'</span><span class="p">,</span> |
| <span class="n">other_kwargs</span><span class="o">=</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.explode"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.explode">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">explode</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">column</span><span class="p">,</span> <span class="n">ignore_index</span><span class="p">):</span> |
| <span class="c1"># ignoring the index will not preserve it</span> |
| <span class="n">preserves</span> <span class="o">=</span> <span class="p">(</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()</span> <span class="k">if</span> <span class="n">ignore_index</span> |
| <span class="k">else</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">())</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'explode'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span><span class="n">column</span><span class="p">,</span> <span class="n">ignore_index</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">preserves</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.insert"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.insert">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">insert</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``value`` cannot be a ``List`` because aligning it with this</span> |
| <span class="sd"> DeferredDataFrame is order-sensitive."""</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"insert(value=list) is not supported because it joins the input "</span> |
| <span class="s2">"list to the deferred DataFrame based on the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">):</span> |
| <span class="n">value</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ConstantExpression</span><span class="p">(</span><span class="n">value</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">func_zip</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="n">value</span><span class="o">=</span><span class="n">value</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">df</span> |
| |
| <span class="n">inserted</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'insert'</span><span class="p">,</span> |
| <span class="n">func_zip</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">value</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">def</span> <span class="nf">func_elementwise</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="n">value</span><span class="o">=</span><span class="n">value</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">df</span> |
| <span class="n">inserted</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'insert'</span><span class="p">,</span> |
| <span class="n">func_elementwise</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> <span class="o">=</span> <span class="n">inserted</span><span class="o">.</span><span class="n">_expr</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.from_dict"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.from_dict">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">from_dict</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ConstantExpression</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_dict</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></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.from_records"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.from_records">[docs]</a> <span class="nd">@staticmethod</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">from_records</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ConstantExpression</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="o">*</span><span class="n">args</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">)))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.duplicated"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.duplicated">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">duplicated</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keep</span><span class="p">,</span> <span class="n">subset</span><span class="p">):</span> |
| <span class="sd">"""Only ``keep=False`` and ``keep="any"`` are supported. Other values of</span> |
| <span class="sd"> ``keep`` make this an order-sensitive operation. Note ``keep="any"`` is</span> |
| <span class="sd"> a Beam-specific option that guarantees only one duplicate will be kept, but</span> |
| <span class="sd"> unlike ``"first"`` and ``"last"`` it makes no guarantees about _which_</span> |
| <span class="sd"> duplicate element is kept."""</span> |
| <span class="c1"># TODO(BEAM-12074): Document keep="any"</span> |
| <span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s1">'any'</span><span class="p">:</span> |
| <span class="n">keep</span> <span class="o">=</span> <span class="s1">'first'</span> |
| <span class="k">elif</span> <span class="n">keep</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">False</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"duplicated(keep=</span><span class="si">{</span><span class="n">keep</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"sensitive to the order of the data. Only keep=False and "</span> |
| <span class="s2">"keep=</span><span class="se">\"</span><span class="s2">any</span><span class="se">\"</span><span class="s2"> are supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="n">by</span> <span class="o">=</span> <span class="n">subset</span> <span class="ow">or</span> <span class="nb">list</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">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">by</span><span class="p">)</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">df</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">df</span><span class="o">.</span><span class="n">duplicated</span><span class="p">(</span><span class="n">keep</span><span class="o">=</span><span class="n">keep</span><span class="p">,</span> <span class="n">subset</span><span class="o">=</span><span class="n">subset</span><span class="p">),</span> |
| <span class="n">columns</span><span class="o">=</span><span class="p">[</span><span class="kc">None</span><span class="p">]))[</span><span class="kc">None</span><span class="p">]</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="n">by</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.drop_duplicates"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.drop_duplicates">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">drop_duplicates</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keep</span><span class="p">,</span> <span class="n">subset</span><span class="p">,</span> <span class="n">ignore_index</span><span class="p">):</span> |
| <span class="sd">"""Only ``keep=False`` and ``keep="any"`` are supported. Other values of</span> |
| <span class="sd"> ``keep`` make this an order-sensitive operation. Note ``keep="any"`` is</span> |
| <span class="sd"> a Beam-specific option that guarantees only one duplicate will be kept, but</span> |
| <span class="sd"> unlike ``"first"`` and ``"last"`` it makes no guarantees about _which_</span> |
| <span class="sd"> duplicate element is kept."""</span> |
| <span class="c1"># TODO(BEAM-12074): Document keep="any"</span> |
| <span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s1">'any'</span><span class="p">:</span> |
| <span class="n">keep</span> <span class="o">=</span> <span class="s1">'first'</span> |
| <span class="k">elif</span> <span class="n">keep</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">False</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"drop_duplicates(keep=</span><span class="si">{</span><span class="n">keep</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"sensitive to the order of the data. Only keep=False and "</span> |
| <span class="s2">"keep=</span><span class="se">\"</span><span class="s2">any</span><span class="se">\"</span><span class="s2"> are supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">ignore_index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">False</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"drop_duplicates(ignore_index=False) is not supported because it "</span> |
| <span class="s2">"requires generating a new index that is sensitive to the order of "</span> |
| <span class="s2">"the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="n">by</span> <span class="o">=</span> <span class="n">subset</span> <span class="ow">or</span> <span class="nb">list</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">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">by</span><span class="p">)</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">drop_duplicates</span><span class="p">(</span><span class="n">keep</span><span class="o">=</span><span class="n">keep</span><span class="p">,</span> <span class="n">subset</span><span class="o">=</span><span class="n">subset</span><span class="p">))</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="n">by</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.aggregate"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.aggregate">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">aggregate</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">axis</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="c1"># We have specialized implementations for these.</span> |
| <span class="k">if</span> <span class="n">func</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'quantile'</span><span class="p">,):</span> |
| <span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">)(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="c1"># In pandas<1.3.0, maps to a property, args are ignored</span> |
| <span class="k">if</span> <span class="n">func</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'size'</span><span class="p">,)</span> <span class="ow">and</span> <span class="n">PD_VERSION</span> <span class="o"><</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="mi">3</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">)</span> |
| |
| <span class="c1"># We also have specialized distributed implementations for these. They only</span> |
| <span class="c1"># support axis=0 (implicitly) though. axis=1 should fall through</span> |
| <span class="k">if</span> <span class="n">func</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'corr'</span><span class="p">,</span> <span class="s1">'cov'</span><span class="p">)</span> <span class="ow">and</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">)(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># Aggregate across all elements by first aggregating across columns,</span> |
| <span class="c1"># then across rows.</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="nb">dict</span><span class="p">(</span><span class="n">kwargs</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">1</span><span class="p">))</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span> |
| <span class="n">func</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="nb">dict</span><span class="p">(</span><span class="n">kwargs</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">0</span><span class="p">))</span> |
| <span class="k">elif</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="c1"># This is an easy elementwise aggregation.</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'aggregate'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">1</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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="c1"># For this corner case, just colocate everything.</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'aggregate'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># In the general case, we will compute the aggregation of each column</span> |
| <span class="c1"># separately, then recombine.</span> |
| |
| <span class="c1"># First, handle any kwargs that cause a projection, by eagerly generating</span> |
| <span class="c1"># the proxy, and only including the columns that are in the output.</span> |
| <span class="n">PROJECT_KWARGS</span> <span class="o">=</span> <span class="p">(</span><span class="s1">'numeric_only'</span><span class="p">,</span> <span class="s1">'bool_only'</span><span class="p">,</span> <span class="s1">'include'</span><span class="p">,</span> <span class="s1">'exclude'</span><span class="p">)</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">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">projected</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="nb">list</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="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">projected</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</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="p">)]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">projected</span> <span class="o">=</span> <span class="bp">self</span> |
| |
| <span class="n">nonnumeric_columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">name</span> <span class="k">for</span> <span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">dtype</span><span class="p">)</span> <span class="ow">in</span> <span class="n">projected</span><span class="o">.</span><span class="n">dtypes</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="k">if</span> <span class="ow">not</span> |
| <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">dtypes</span><span class="o">.</span><span class="n">common</span><span class="o">.</span><span class="n">is_numeric_dtype</span><span class="p">(</span><span class="n">dtype</span><span class="p">)]</span> |
| |
| <span class="k">if</span> <span class="n">_is_numeric</span><span class="p">(</span><span class="n">func</span><span class="p">)</span> <span class="ow">and</span> <span class="n">nonnumeric_columns</span><span class="p">:</span> |
| <span class="k">if</span> <span class="s1">'numeric_only'</span> <span class="ow">in</span> <span class="n">kwargs</span> <span class="ow">and</span> <span class="n">kwargs</span><span class="p">[</span><span class="s1">'numeric_only'</span><span class="p">]</span> <span class="ow">is</span> <span class="kc">False</span><span class="p">:</span> |
| <span class="c1"># User has opted in to execution with non-numeric columns, they</span> |
| <span class="c1"># will accept runtime errors</span> |
| <span class="k">pass</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"Numeric aggregation (</span><span class="si">{</span><span class="n">func</span><span class="si">!r}</span><span class="s2">) on a DataFrame containing "</span> |
| <span class="sa">f</span><span class="s2">"non-numeric columns (</span><span class="si">{</span><span class="o">*</span><span class="n">nonnumeric_columns</span><span class="p">,</span><span class="si">!r}</span><span class="s2"> is not "</span> |
| <span class="s2">"supported, unless `numeric_only=` is specified.</span><span class="se">\n</span><span class="s2">"</span> |
| <span class="s2">"Use `numeric_only=True` to only aggregate over numeric "</span> |
| <span class="s2">"columns.</span><span class="se">\n</span><span class="s2">Use `numeric_only=False` to aggregate over all "</span> |
| <span class="s2">"columns. Note this is not recommended, as it could result in "</span> |
| <span class="s2">"execution time errors."</span><span class="p">)</span> |
| |
| <span class="k">for</span> <span class="n">key</span> <span class="ow">in</span> <span class="n">PROJECT_KWARGS</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">key</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="p">:</span> |
| <span class="n">kwargs</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span> |
| <span class="n">col_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">projected</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span> |
| <span class="n">func_by_col</span> <span class="o">=</span> <span class="p">{</span><span class="n">col</span><span class="p">:</span> <span class="n">func</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">col_names</span><span class="p">}</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">func_by_col</span> <span class="o">=</span> <span class="n">func</span> |
| <span class="n">col_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">func</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| <span class="n">aggregated_cols</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">has_lists</span> <span class="o">=</span> <span class="nb">any</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">f</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="k">for</span> <span class="n">f</span> <span class="ow">in</span> <span class="n">func_by_col</span><span class="o">.</span><span class="n">values</span><span class="p">())</span> |
| <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">col_names</span><span class="p">:</span> |
| <span class="n">funcs</span> <span class="o">=</span> <span class="n">func_by_col</span><span class="p">[</span><span class="n">col</span><span class="p">]</span> |
| <span class="k">if</span> <span class="n">has_lists</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">funcs</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="c1"># If any of the columns do multiple aggregations, they all must use</span> |
| <span class="c1"># "list" style output</span> |
| <span class="n">funcs</span> <span class="o">=</span> <span class="p">[</span><span class="n">funcs</span><span class="p">]</span> |
| <span class="n">aggregated_cols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">projected</span><span class="p">[</span><span class="n">col</span><span class="p">]</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">funcs</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="c1"># The final shape is different depending on whether any of the columns</span> |
| <span class="c1"># were aggregated by a list of aggregators.</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</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">Series</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'join_aggregate'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="o">*</span><span class="n">cols</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="p">{</span><span class="n">col</span><span class="p">:</span> <span class="n">value</span> <span class="k">for</span> <span class="n">col</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">col_names</span><span class="p">,</span> <span class="n">cols</span><span class="p">)}),</span> |
| <span class="p">[</span><span class="n">col</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">aggregated_cols</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</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">DataFrame</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'join_aggregate'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="o">*</span><span class="n">cols</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="p">{</span><span class="n">col</span><span class="p">:</span> <span class="n">value</span> <span class="k">for</span> <span class="n">col</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">col_names</span><span class="p">,</span> <span class="n">cols</span><span class="p">)}),</span> |
| <span class="p">[</span><span class="n">col</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">aggregated_cols</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">AssertionError</span><span class="p">(</span><span class="s2">"Unexpected proxy type for "</span> |
| <span class="sa">f</span><span class="s2">"DataFrame.aggregate!: proxy=</span><span class="si">{</span><span class="n">proxy</span><span class="si">!r}</span><span class="s2">, "</span> |
| <span class="sa">f</span><span class="s2">"type(proxy)=</span><span class="si">{</span><span class="nb">type</span><span class="p">(</span><span class="n">proxy</span><span class="p">)</span><span class="si">!r}</span><span class="s2">"</span><span class="p">)</span></div> |
| |
| <span class="n">agg</span> <span class="o">=</span> <span class="n">aggregate</span> |
| |
| <span class="n">applymap</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'applymap'</span><span class="p">,</span> <span class="n">base</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">add_prefix</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'add_prefix'</span><span class="p">,</span> <span class="n">base</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">add_suffix</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'add_suffix'</span><span class="p">,</span> <span class="n">base</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">memory_usage</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'memory_usage'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">info</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'info'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| |
| <div class="viewcode-block" id="DeferredDataFrame.clip"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.clip">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">clip</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``lower`` and ``upper`` must be :class:`DeferredSeries` instances, or</span> |
| <span class="sd"> constants. Array-like arguments are not supported because they are</span> |
| <span class="sd"> order-sensitive."""</span> |
| |
| <span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">arg</span><span class="p">,</span> <span class="kc">None</span><span class="p">),</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'upper'</span><span class="p">,</span> <span class="s1">'lower'</span><span class="p">))</span> <span class="ow">and</span> <span class="n">axis</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"axis must be 'index' when upper and/or lower are a DeferredFrame"</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'clip'</span><span class="p">,</span> <span class="n">base</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="bp">self</span><span class="p">,</span> |
| <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.corr"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.corr">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">corr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">method</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">):</span> |
| <span class="sd">"""Only ``method="pearson"`` can be parallelized. Other methods require</span> |
| <span class="sd"> collecting all data on a single worker (see</span> |
| <span class="sd"> https://s.apache.org/dataframe-non-parallel-operations for details).</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="n">method</span> <span class="o">==</span> <span class="s1">'pearson'</span><span class="p">:</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">corr</span><span class="p">()</span> |
| <span class="n">columns</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">columns</span><span class="p">)</span> |
| <span class="n">args</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">arg_indices</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">col1</span><span class="p">,</span> <span class="n">col2</span> <span class="ow">in</span> <span class="n">itertools</span><span class="o">.</span><span class="n">combinations</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="mi">2</span><span class="p">):</span> |
| <span class="n">arg_indices</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">col1</span><span class="p">,</span> <span class="n">col2</span><span class="p">))</span> |
| <span class="n">args</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="bp">self</span><span class="p">[</span><span class="n">col1</span><span class="p">]</span><span class="o">.</span><span class="n">corr</span><span class="p">(</span><span class="bp">self</span><span class="p">[</span><span class="n">col2</span><span class="p">],</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> |
| <span class="n">min_periods</span><span class="o">=</span><span class="n">min_periods</span><span class="p">))</span> |
| <span class="k">def</span> <span class="nf">fill_matrix</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">):</span> |
| <span class="n">data</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">dict</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">columns</span><span class="p">:</span> |
| <span class="n">data</span><span class="p">[</span><span class="n">col</span><span class="p">][</span><span class="n">col</span><span class="p">]</span> <span class="o">=</span> <span class="mf">1.0</span> |
| <span class="k">for</span> <span class="n">ix</span><span class="p">,</span> <span class="p">(</span><span class="n">col1</span><span class="p">,</span> <span class="n">col2</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">arg_indices</span><span class="p">):</span> |
| <span class="n">data</span><span class="p">[</span><span class="n">col1</span><span class="p">][</span><span class="n">col2</span><span class="p">]</span> <span class="o">=</span> <span class="n">data</span><span class="p">[</span><span class="n">col2</span><span class="p">][</span><span class="n">col1</span><span class="p">]</span> <span class="o">=</span> <span class="n">args</span><span class="p">[</span><span class="n">ix</span><span class="p">]</span> |
| <span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">data</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="n">index</span><span class="o">=</span><span class="n">columns</span><span class="p">)</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'fill_matrix'</span><span class="p">,</span> |
| <span class="n">fill_matrix</span><span class="p">,</span> |
| <span class="p">[</span><span class="n">arg</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">args</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">))</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">reason</span> <span class="o">=</span> <span class="p">(</span><span class="sa">f</span><span class="s2">"Encountered corr(method=</span><span class="si">{</span><span class="n">method</span><span class="si">!r}</span><span class="s2">) which cannot be "</span> |
| <span class="s2">"parallelized. Only corr(method='pearson') is currently "</span> |
| <span class="s2">"parallelizable."</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'corr'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">corr</span><span class="p">(</span><span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="n">min_periods</span><span class="o">=</span><span class="n">min_periods</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="n">reason</span><span class="p">)))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.cov"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.cov">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">cov</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">,</span> <span class="n">ddof</span><span class="p">):</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">corr</span><span class="p">()</span> |
| <span class="n">columns</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">columns</span><span class="p">)</span> |
| <span class="n">args</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="n">arg_indices</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">columns</span><span class="p">:</span> |
| <span class="n">arg_indices</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">col</span><span class="p">,</span> <span class="n">col</span><span class="p">))</span> |
| <span class="n">std</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="n">col</span><span class="p">]</span><span class="o">.</span><span class="n">std</span><span class="p">(</span><span class="n">ddof</span><span class="p">)</span> |
| <span class="n">args</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">std</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="o">*</span><span class="n">x</span><span class="p">,</span> <span class="s1">'square'</span><span class="p">))</span> |
| <span class="k">for</span> <span class="n">ix</span><span class="p">,</span> <span class="n">col1</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">columns</span><span class="p">):</span> |
| <span class="k">for</span> <span class="n">col2</span> <span class="ow">in</span> <span class="n">columns</span><span class="p">[</span><span class="n">ix</span><span class="o">+</span><span class="mi">1</span><span class="p">:]:</span> |
| <span class="n">arg_indices</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">col1</span><span class="p">,</span> <span class="n">col2</span><span class="p">))</span> |
| <span class="c1"># Note that this set may be different for each pair.</span> |
| <span class="n">no_na</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="bp">self</span><span class="p">[</span><span class="n">col1</span><span class="p">]</span><span class="o">.</span><span class="n">notna</span><span class="p">()</span> <span class="o">&</span> <span class="bp">self</span><span class="p">[</span><span class="n">col2</span><span class="p">]</span><span class="o">.</span><span class="n">notna</span><span class="p">()]</span> |
| <span class="n">args</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">no_na</span><span class="p">[</span><span class="n">col1</span><span class="p">]</span><span class="o">.</span><span class="n">_cov_aligned</span><span class="p">(</span><span class="n">no_na</span><span class="p">[</span><span class="n">col2</span><span class="p">],</span> <span class="n">min_periods</span><span class="p">,</span> <span class="n">ddof</span><span class="p">))</span> |
| <span class="k">def</span> <span class="nf">fill_matrix</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">):</span> |
| <span class="n">data</span> <span class="o">=</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">dict</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">ix</span><span class="p">,</span> <span class="p">(</span><span class="n">col1</span><span class="p">,</span> <span class="n">col2</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">arg_indices</span><span class="p">):</span> |
| <span class="n">data</span><span class="p">[</span><span class="n">col1</span><span class="p">][</span><span class="n">col2</span><span class="p">]</span> <span class="o">=</span> <span class="n">data</span><span class="p">[</span><span class="n">col2</span><span class="p">][</span><span class="n">col1</span><span class="p">]</span> <span class="o">=</span> <span class="n">args</span><span class="p">[</span><span class="n">ix</span><span class="p">]</span> |
| <span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">data</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="n">index</span><span class="o">=</span><span class="n">columns</span><span class="p">)</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'fill_matrix'</span><span class="p">,</span> |
| <span class="n">fill_matrix</span><span class="p">,</span> |
| <span class="p">[</span><span class="n">arg</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">args</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.corrwith"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.corrwith">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">corrwith</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">drop</span><span class="p">,</span> <span class="n">method</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_elementwise</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">corrwith</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="n">drop</span><span class="p">,</span> |
| <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">),</span> |
| <span class="s1">'corrwith'</span><span class="p">,</span> |
| <span class="n">other_args</span><span class="o">=</span><span class="p">(</span><span class="n">other</span><span class="p">,))</span> |
| |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="n">other</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ConstantExpression</span><span class="p">(</span><span class="n">other</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DeferredSeries</span><span class="p">):</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">corrwith</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">(),</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> |
| <span class="n">drop</span><span class="o">=</span><span class="n">drop</span><span class="p">,</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">)</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">other</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">join</span><span class="o">=</span><span class="s1">'inner'</span><span class="p">)</span> |
| <span class="n">col_names</span> <span class="o">=</span> <span class="n">proxy</span><span class="o">.</span><span class="n">index</span> |
| <span class="n">other_cols</span> <span class="o">=</span> <span class="p">[</span><span class="n">other</span><span class="p">]</span> <span class="o">*</span> <span class="nb">len</span><span class="p">(</span><span class="n">col_names</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DeferredDataFrame</span><span class="p">):</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">corrwith</span><span class="p">(</span> |
| <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">(),</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="n">drop</span><span class="p">)</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">other</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">align</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">join</span><span class="o">=</span><span class="s1">'inner'</span><span class="p">)</span> |
| <span class="n">col_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span> |
| <span class="nb">set</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="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span> |
| <span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="n">proxy</span><span class="o">.</span><span class="n">index</span><span class="p">))</span> |
| <span class="n">other_cols</span> <span class="o">=</span> <span class="p">[</span><span class="n">other</span><span class="p">[</span><span class="n">col_name</span><span class="p">]</span> <span class="k">for</span> <span class="n">col_name</span> <span class="ow">in</span> <span class="n">col_names</span><span class="p">]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Raise the right error.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">corrwith</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">(),</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="n">drop</span><span class="p">,</span> |
| <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">)</span> |
| |
| <span class="c1"># Just in case something else becomes valid.</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">'corrwith(</span><span class="si">%s</span><span class="s1">)'</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">))</span> |
| |
| <span class="c1"># Generate expressions to compute the actual correlations.</span> |
| <span class="n">corrs</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="bp">self</span><span class="p">[</span><span class="n">col_name</span><span class="p">]</span><span class="o">.</span><span class="n">corr</span><span class="p">(</span><span class="n">other_col</span><span class="p">,</span> <span class="n">method</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">col_name</span><span class="p">,</span> <span class="n">other_col</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">col_names</span><span class="p">,</span> <span class="n">other_cols</span><span class="p">)]</span> |
| |
| <span class="c1"># Combine the results</span> |
| <span class="k">def</span> <span class="nf">fill_dataframe</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">):</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">proxy</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">deep</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">col</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">proxy</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="n">args</span><span class="p">):</span> |
| <span class="n">result</span><span class="p">[</span><span class="n">col</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span> |
| <span class="k">return</span> <span class="n">result</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'fill_dataframe'</span><span class="p">,</span> |
| <span class="n">fill_dataframe</span><span class="p">,</span> |
| <span class="p">[</span><span class="n">corr</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">corr</span> <span class="ow">in</span> <span class="n">corrs</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">))</span></div> |
| |
| <span class="n">cummax</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'cummax'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cummin</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'cummin'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cumprod</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'cumprod'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cumsum</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'cumsum'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="c1"># TODO(BEAM-12071): Consider adding an order-insensitive implementation for</span> |
| <span class="c1"># diff that relies on the index</span> |
| <span class="n">diff</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'diff'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">interpolate</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'interpolate'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| |
| <span class="n">pct_change</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'pct_change'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">asof</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'asof'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">first_valid_index</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'first_valid_index'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">last_valid_index</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'last_valid_index'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">iat</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'iat'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">))</span> |
| |
| <span class="n">lookup</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'lookup'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'deprecated'</span><span class="p">)</span> |
| |
| <span class="n">head</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'head'</span><span class="p">,</span> |
| <span class="n">explanation</span><span class="o">=</span><span class="n">_PEEK_METHOD_EXPLANATION</span><span class="p">)</span> |
| <span class="n">tail</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'tail'</span><span class="p">,</span> |
| <span class="n">explanation</span><span class="o">=</span><span class="n">_PEEK_METHOD_EXPLANATION</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.sample"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.sample">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">sample</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">frac</span><span class="p">,</span> <span class="n">replace</span><span class="p">,</span> <span class="n">weights</span><span class="p">,</span> <span class="n">random_state</span><span class="p">,</span> <span class="n">axis</span><span class="p">):</span> |
| <span class="sd">"""When ``axis='index'``, only ``n`` and/or ``weights`` may be specified.</span> |
| <span class="sd"> ``frac``, ``random_state``, and ``replace=True`` are not yet supported.</span> |
| <span class="sd"> See `BEAM-12476 <https://issues.apache.org/jira/BEAM-12476>`_.</span> |
| |
| <span class="sd"> Note that pandas will raise an error if ``n`` is larger than the length</span> |
| <span class="sd"> of the dataset, while the Beam DataFrame API will simply return the full</span> |
| <span class="sd"> dataset in that case.</span> |
| |
| <span class="sd"> sample is fully supported for axis='columns'."""</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="c1"># Sampling on axis=columns just means projecting random columns</span> |
| <span class="c1"># Eagerly generate proxy to determine the set of columns at construction</span> |
| <span class="c1"># time</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">sample</span><span class="p">(</span><span class="n">n</span><span class="o">=</span><span class="n">n</span><span class="p">,</span> <span class="n">frac</span><span class="o">=</span><span class="n">frac</span><span class="p">,</span> <span class="n">replace</span><span class="o">=</span><span class="n">replace</span><span class="p">,</span> |
| <span class="n">weights</span><span class="o">=</span><span class="n">weights</span><span class="p">,</span> |
| <span class="n">random_state</span><span class="o">=</span><span class="n">random_state</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">)</span> |
| <span class="c1"># Then do the projection</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="nb">list</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="c1"># axis='index'</span> |
| <span class="k">if</span> <span class="n">frac</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">random_state</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">replace</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"When axis=</span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">, only n and/or weights may be specified. "</span> |
| <span class="s2">"frac, random_state, and replace=True are not yet supported "</span> |
| <span class="sa">f</span><span class="s2">"(got frac=</span><span class="si">{</span><span class="n">frac</span><span class="si">!r}</span><span class="s2">, random_state=</span><span class="si">{</span><span class="n">random_state</span><span class="si">!r}</span><span class="s2">, "</span> |
| <span class="sa">f</span><span class="s2">"replace=</span><span class="si">{</span><span class="n">replace</span><span class="si">!r}</span><span class="s2">). See BEAM-12476."</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">n</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">n</span> <span class="o">=</span> <span class="mi">1</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">weights</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span> |
| <span class="n">weights</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="n">weights</span><span class="p">]</span> |
| |
| <span class="n">tmp_weight_column_name</span> <span class="o">=</span> <span class="s2">"___Beam_DataFrame_weights___"</span> |
| |
| <span class="k">if</span> <span class="n">weights</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">self_with_randomized_weights</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'randomized_weights'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">assign</span><span class="p">(</span><span class="o">**</span><span class="p">{</span><span class="n">tmp_weight_column_name</span><span class="p">:</span> |
| <span class="n">np</span><span class="o">.</span><span class="n">random</span><span class="o">.</span><span class="n">rand</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">df</span><span class="p">))}),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># See "Fast Parallel Weighted Random Sampling" by Efraimidis and Spirakis</span> |
| <span class="c1"># https://www.cti.gr/images_gr/reports/99-06-02.ps</span> |
| <span class="k">def</span> <span class="nf">assign_randomized_weights</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="n">weights</span><span class="p">):</span> |
| <span class="n">non_zero_weights</span> <span class="o">=</span> <span class="p">(</span><span class="n">weights</span> <span class="o">></span> <span class="mi">0</span><span class="p">)</span> <span class="o">|</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="nb">bool</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="p">)</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">non_zero_weights</span><span class="p">]</span> |
| <span class="n">weights</span> <span class="o">=</span> <span class="n">weights</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">non_zero_weights</span><span class="p">]</span> |
| <span class="n">random_weights</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">log</span><span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">random</span><span class="o">.</span><span class="n">rand</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">weights</span><span class="p">)))</span> <span class="o">/</span> <span class="n">weights</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">assign</span><span class="p">(</span><span class="o">**</span><span class="p">{</span><span class="n">tmp_weight_column_name</span><span class="p">:</span> <span class="n">random_weights</span><span class="p">})</span> |
| <span class="n">self_with_randomized_weights</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'randomized_weights'</span><span class="p">,</span> |
| <span class="n">assign_randomized_weights</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">weights</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="k">return</span> <span class="n">self_with_randomized_weights</span><span class="o">.</span><span class="n">nlargest</span><span class="p">(</span> |
| <span class="n">n</span><span class="o">=</span><span class="n">n</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">tmp_weight_column_name</span><span class="p">,</span> <span class="n">keep</span><span class="o">=</span><span class="s1">'any'</span><span class="p">)</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span> |
| <span class="n">tmp_weight_column_name</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">1</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.dot"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.dot">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">dot</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span> |
| <span class="c1"># We want to broadcast the right hand side to all partitions of the left.</span> |
| <span class="c1"># This is OK, as its index must be the same size as the columns set of self,</span> |
| <span class="c1"># so cannot be too large.</span> |
| <span class="k">class</span> <span class="nc">AsScalar</span><span class="p">(</span><span class="nb">object</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">value</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">value</span> <span class="o">=</span> <span class="n">value</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">():</span> |
| <span class="n">side</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'as_scalar'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">AsScalar</span><span class="p">(</span><span class="n">df</span><span class="p">),</span> |
| <span class="p">[</span><span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| <span class="k">else</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="nb">range</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">other</span><span class="p">[</span><span class="mi">0</span><span class="p">])))</span> |
| <span class="n">side</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ConstantExpression</span><span class="p">(</span><span class="n">AsScalar</span><span class="p">(</span><span class="n">other</span><span class="p">))</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'dot'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">left</span><span class="p">,</span> <span class="n">right</span><span class="p">:</span> <span class="n">left</span> <span class="o">@</span> <span class="n">right</span><span class="o">.</span><span class="n">value</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">side</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">))</span></div> |
| |
| <span class="fm">__matmul__</span> <span class="o">=</span> <span class="n">dot</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.mode"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.mode">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">mode</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">0</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="sd">"""mode with axis="columns" is not implemented because it produces</span> |
| <span class="sd"> non-deferred columns.</span> |
| |
| <span class="sd"> mode with axis="index" is not currently parallelizable. An approximate,</span> |
| <span class="sd"> parallelizable implementation of mode may be added in the future</span> |
| <span class="sd"> (`BEAM-12181 <https://issues.apache.org/jira/BEAM-12181>`_)."""</span> |
| |
| <span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">1</span> <span class="ow">or</span> <span class="n">axis</span> <span class="o">==</span> <span class="s1">'columns'</span><span class="p">:</span> |
| <span class="c1"># Number of columns is max(number mode values for each row), so we can't</span> |
| <span class="c1"># determine how many there will be before looking at the data.</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"mode(axis=columns) is not supported because it produces a variable "</span> |
| <span class="s2">"number of columns depending on the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'mode'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">mode</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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="c1">#TODO(BEAM-12181): Can we add an approximate implementation?</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"mode(axis='index') cannot currently be parallelized. See "</span> |
| <span class="s2">"BEAM-12181 tracking the possble addition of an approximate, "</span> |
| <span class="s2">"parallelizable implementation of mode."</span> |
| <span class="p">)),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.dropna"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.dropna">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">dropna</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""dropna with axis="columns" specified cannot be parallelized."""</span> |
| <span class="c1"># TODO(robertwb): This is a common pattern. Generalize?</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"dropna(axis=1) cannot currently be parallelized. It requires "</span> |
| <span class="s2">"checking all values in each column for NaN values, to determine "</span> |
| <span class="s2">"if that column should be dropped."</span> |
| <span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'dropna'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">dropna</span><span class="p">(</span><span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires_partition_by</span><span class="p">))</span></div> |
| |
| <span class="k">def</span> <span class="nf">_eval_or_query</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">expr</span><span class="p">,</span> <span class="n">inplace</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">for</span> <span class="n">key</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'local_dict'</span><span class="p">,</span> <span class="s1">'global_dict'</span><span class="p">,</span> <span class="s1">'level'</span><span class="p">,</span> <span class="s1">'target'</span><span class="p">,</span> <span class="s1">'resolvers'</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">key</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="sa">f</span><span class="s2">"Setting '</span><span class="si">{</span><span class="n">key</span><span class="si">}</span><span class="s2">' is not yet supported"</span><span class="p">)</span> |
| |
| <span class="c1"># look for '@<py identifier>'</span> |
| <span class="k">if</span> <span class="n">re</span><span class="o">.</span><span class="n">search</span><span class="p">(</span><span class="sa">r</span><span class="s1">'\@[^\d\W]\w*'</span><span class="p">,</span> <span class="n">expr</span><span class="p">,</span> <span class="n">re</span><span class="o">.</span><span class="n">UNICODE</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">"Accessing locals with @ is not yet supported "</span> |
| <span class="s2">"(BEAM-11202)"</span><span class="p">)</span> |
| |
| <span class="n">result_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="n">name</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="n">name</span><span class="p">)(</span><span class="n">expr</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| |
| <span class="k">if</span> <span class="n">inplace</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> <span class="o">=</span> <span class="n">result_expr</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span><span class="n">result_expr</span><span class="p">)</span> |
| |
| |
| <div class="viewcode-block" id="DeferredDataFrame.eval"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.eval">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">eval</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">expr</span><span class="p">,</span> <span class="n">inplace</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Accessing local variables with ``@<varname>`` is not yet supported</span> |
| <span class="sd"> (`BEAM-11202 <https://issues.apache.org/jira/browse/BEAM-11202>`_).</span> |
| |
| <span class="sd"> Arguments ``local_dict``, ``global_dict``, ``level``, ``target``, and</span> |
| <span class="sd"> ``resolvers`` are not yet supported."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_eval_or_query</span><span class="p">(</span><span class="s1">'eval'</span><span class="p">,</span> <span class="n">expr</span><span class="p">,</span> <span class="n">inplace</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.query"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.query">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">query</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">expr</span><span class="p">,</span> <span class="n">inplace</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Accessing local variables with ``@<varname>`` is not yet supported</span> |
| <span class="sd"> (`BEAM-11202 <https://issues.apache.org/jira/browse/BEAM-11202>`_).</span> |
| |
| <span class="sd"> Arguments ``local_dict``, ``global_dict``, ``level``, ``target``, and</span> |
| <span class="sd"> ``resolvers`` are not yet supported."""</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_eval_or_query</span><span class="p">(</span><span class="s1">'query'</span><span class="p">,</span> <span class="n">expr</span><span class="p">,</span> <span class="n">inplace</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| <span class="n">isnull</span> <span class="o">=</span> <span class="n">isna</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'isna'</span><span class="p">,</span> <span class="n">base</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">notnull</span> <span class="o">=</span> <span class="n">notna</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'notna'</span><span class="p">,</span> <span class="n">base</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">items</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'items'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">itertuples</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'itertuples'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">iterrows</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'iterrows'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">iteritems</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'iteritems'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_cols_as_temporary_index</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">cols</span><span class="p">,</span> <span class="n">suffix</span><span class="o">=</span><span class="s1">''</span><span class="p">):</span> |
| <span class="n">original_index_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</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">new_index_names</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'__apache_beam_temp_</span><span class="si">%d</span><span class="s1">_</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="n">ix</span><span class="p">,</span> <span class="n">suffix</span><span class="p">)</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">ix</span><span class="p">,</span> <span class="n">_</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">original_index_names</span><span class="p">)]</span> |
| <span class="k">def</span> <span class="nf">reindex</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'reindex'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">rename_axis</span><span class="p">(</span><span class="n">index</span><span class="o">=</span><span class="n">new_index_names</span><span class="p">,</span> <span class="n">copy</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span> |
| <span class="o">.</span><span class="n">reset_index</span><span class="p">()</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">cols</span><span class="p">),</span> |
| <span class="p">[</span><span class="n">df</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">def</span> <span class="nf">revert</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'join_restoreindex'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">reset_index</span><span class="p">()</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">new_index_names</span><span class="p">)</span> |
| <span class="o">.</span><span class="n">rename_axis</span><span class="p">(</span><span class="n">index</span><span class="o">=</span><span class="n">original_index_names</span><span class="p">,</span> <span class="n">copy</span><span class="o">=</span><span class="kc">False</span><span class="p">),</span> |
| <span class="p">[</span><span class="n">df</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">return</span> <span class="n">reindex</span><span class="p">,</span> <span class="n">revert</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.join"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.join">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">join</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">on</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">on</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">reindex</span><span class="p">,</span> <span class="n">revert</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_cols_as_temporary_index</span><span class="p">(</span><span class="n">on</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">revert</span><span class="p">(</span><span class="n">reindex</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">))</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="n">other_is_list</span> <span class="o">=</span> <span class="kc">True</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">other</span> <span class="o">=</span> <span class="p">[</span><span class="n">other</span><span class="p">]</span> |
| <span class="n">other_is_list</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="n">placeholder</span> <span class="o">=</span> <span class="nb">object</span><span class="p">()</span> |
| <span class="n">other_exprs</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">df</span> <span class="ow">in</span> <span class="n">other</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">)]</span> |
| <span class="n">const_others</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">placeholder</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">)</span> <span class="k">else</span> <span class="n">df</span> |
| <span class="k">for</span> <span class="n">df</span> <span class="ow">in</span> <span class="n">other</span><span class="p">]</span> |
| <span class="k">def</span> <span class="nf">fill_placeholders</span><span class="p">(</span><span class="n">values</span><span class="p">):</span> |
| <span class="n">values</span> <span class="o">=</span> <span class="nb">iter</span><span class="p">(</span><span class="n">values</span><span class="p">)</span> |
| <span class="n">filled</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="nb">next</span><span class="p">(</span><span class="n">values</span><span class="p">)</span> <span class="k">if</span> <span class="n">df</span> <span class="ow">is</span> <span class="n">placeholder</span> <span class="k">else</span> <span class="n">df</span> <span class="k">for</span> <span class="n">df</span> <span class="ow">in</span> <span class="n">const_others</span><span class="p">]</span> |
| <span class="k">if</span> <span class="n">other_is_list</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">filled</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">filled</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'join'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> <span class="o">*</span><span class="n">deferred_others</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">join</span><span class="p">(</span> |
| <span class="n">fill_placeholders</span><span class="p">(</span><span class="n">deferred_others</span><span class="p">),</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">]</span> <span class="o">+</span> <span class="n">other_exprs</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.merge"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.merge">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">merge</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">right</span><span class="p">,</span> |
| <span class="n">on</span><span class="p">,</span> |
| <span class="n">left_on</span><span class="p">,</span> |
| <span class="n">right_on</span><span class="p">,</span> |
| <span class="n">left_index</span><span class="p">,</span> |
| <span class="n">right_index</span><span class="p">,</span> |
| <span class="n">suffixes</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""merge is not parallelizable unless ``left_index`` or ``right_index`` is</span> |
| <span class="sd"> ``True`, because it requires generating an entirely new unique index.</span> |
| <span class="sd"> See notes on :meth:`DeferredDataFrame.reset_index`. It is recommended to</span> |
| <span class="sd"> move the join key for one of your columns to the index to avoid this issue.</span> |
| <span class="sd"> For an example see the enrich pipeline in</span> |
| <span class="sd"> :mod:`apache_beam.examples.dataframe.taxiride`.</span> |
| |
| <span class="sd"> ``how="cross"`` is not yet supported.</span> |
| <span class="sd"> """</span> |
| <span class="n">self_proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span> |
| <span class="n">right_proxy</span> <span class="o">=</span> <span class="n">right</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span> |
| <span class="c1"># Validate with a pandas call.</span> |
| <span class="n">_</span> <span class="o">=</span> <span class="n">self_proxy</span><span class="o">.</span><span class="n">merge</span><span class="p">(</span> |
| <span class="n">right_proxy</span><span class="p">,</span> |
| <span class="n">on</span><span class="o">=</span><span class="n">on</span><span class="p">,</span> |
| <span class="n">left_on</span><span class="o">=</span><span class="n">left_on</span><span class="p">,</span> |
| <span class="n">right_on</span><span class="o">=</span><span class="n">right_on</span><span class="p">,</span> |
| <span class="n">left_index</span><span class="o">=</span><span class="n">left_index</span><span class="p">,</span> |
| <span class="n">right_index</span><span class="o">=</span><span class="n">right_index</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'how'</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> <span class="o">==</span> <span class="s1">'cross'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">"cross join is not yet implemented (BEAM-9547)"</span><span class="p">)</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">any</span><span class="p">([</span><span class="n">on</span><span class="p">,</span> <span class="n">left_on</span><span class="p">,</span> <span class="n">right_on</span><span class="p">,</span> <span class="n">left_index</span><span class="p">,</span> <span class="n">right_index</span><span class="p">]):</span> |
| <span class="n">on</span> <span class="o">=</span> <span class="p">[</span><span class="n">col</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">self_proxy</span><span class="o">.</span><span class="n">columns</span> <span class="k">if</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">right_proxy</span><span class="o">.</span><span class="n">columns</span><span class="p">]</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">left_on</span><span class="p">:</span> |
| <span class="n">left_on</span> <span class="o">=</span> <span class="n">on</span> |
| <span class="k">if</span> <span class="n">left_on</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">left_on</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="n">left_on</span> <span class="o">=</span> <span class="p">[</span><span class="n">left_on</span><span class="p">]</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">right_on</span><span class="p">:</span> |
| <span class="n">right_on</span> <span class="o">=</span> <span class="n">on</span> |
| <span class="k">if</span> <span class="n">right_on</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">right_on</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="n">right_on</span> <span class="o">=</span> <span class="p">[</span><span class="n">right_on</span><span class="p">]</span> |
| |
| <span class="k">if</span> <span class="n">left_index</span><span class="p">:</span> |
| <span class="n">indexed_left</span> <span class="o">=</span> <span class="bp">self</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">indexed_left</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">left_on</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">right_index</span><span class="p">:</span> |
| <span class="n">indexed_right</span> <span class="o">=</span> <span class="n">right</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">indexed_right</span> <span class="o">=</span> <span class="n">right</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">right_on</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">left_on</span> <span class="ow">and</span> <span class="n">right_on</span><span class="p">:</span> |
| <span class="n">common_cols</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">left_on</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="n">right_on</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">common_cols</span><span class="p">):</span> |
| <span class="c1"># When merging on the same column name from both dfs, we need to make</span> |
| <span class="c1"># sure only one df has the column. Otherwise we end up with</span> |
| <span class="c1"># two duplicate columns, one with lsuffix and one with rsuffix.</span> |
| <span class="c1"># It's safe to drop from either because the data has already been duped</span> |
| <span class="c1"># to the index.</span> |
| <span class="n">indexed_right</span> <span class="o">=</span> <span class="n">indexed_right</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">columns</span><span class="o">=</span><span class="n">common_cols</span><span class="p">)</span> |
| |
| |
| <span class="n">merged</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'merge'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">left</span><span class="p">,</span> <span class="n">right</span><span class="p">:</span> <span class="n">left</span><span class="o">.</span><span class="n">merge</span><span class="p">(</span><span class="n">right</span><span class="p">,</span> |
| <span class="n">left_index</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">right_index</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">suffixes</span><span class="o">=</span><span class="n">suffixes</span><span class="p">,</span> |
| <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="n">indexed_left</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">indexed_right</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()))</span> |
| |
| <span class="k">if</span> <span class="n">left_index</span> <span class="ow">or</span> <span class="n">right_index</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">merged</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">merged</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.nlargest"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.nlargest">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">nlargest</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keep</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Only ``keep=False`` and ``keep="any"`` are supported. Other values of</span> |
| <span class="sd"> ``keep`` make this an order-sensitive operation. Note ``keep="any"`` is</span> |
| <span class="sd"> a Beam-specific option that guarantees only one duplicate will be kept, but</span> |
| <span class="sd"> unlike ``"first"`` and ``"last"`` it makes no guarantees about _which_</span> |
| <span class="sd"> duplicate element is kept."""</span> |
| <span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s1">'any'</span><span class="p">:</span> |
| <span class="n">keep</span> <span class="o">=</span> <span class="s1">'first'</span> |
| <span class="k">elif</span> <span class="n">keep</span> <span class="o">!=</span> <span class="s1">'all'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"nlargest(keep=</span><span class="si">{</span><span class="n">keep</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"order sensitive. Only keep=</span><span class="se">\"</span><span class="s2">all</span><span class="se">\"</span><span class="s2"> is supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="s1">'keep'</span><span class="p">]</span> <span class="o">=</span> <span class="n">keep</span> |
| <span class="n">per_partition</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'nlargest-per-partition'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">nlargest</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'nlargest'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">nlargest</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="n">per_partition</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.nsmallest"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.nsmallest">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">nsmallest</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keep</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""Only ``keep=False`` and ``keep="any"`` are supported. Other values of</span> |
| <span class="sd"> ``keep`` make this an order-sensitive operation. Note ``keep="any"`` is</span> |
| <span class="sd"> a Beam-specific option that guarantees only one duplicate will be kept, but</span> |
| <span class="sd"> unlike ``"first"`` and ``"last"`` it makes no guarantees about _which_</span> |
| <span class="sd"> duplicate element is kept."""</span> |
| <span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s1">'any'</span><span class="p">:</span> |
| <span class="n">keep</span> <span class="o">=</span> <span class="s1">'first'</span> |
| <span class="k">elif</span> <span class="n">keep</span> <span class="o">!=</span> <span class="s1">'all'</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"nsmallest(keep=</span><span class="si">{</span><span class="n">keep</span><span class="si">!r}</span><span class="s2">) is not supported because it is "</span> |
| <span class="s2">"order sensitive. Only keep=</span><span class="se">\"</span><span class="s2">all</span><span class="se">\"</span><span class="s2"> is supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="s1">'keep'</span><span class="p">]</span> <span class="o">=</span> <span class="n">keep</span> |
| <span class="n">per_partition</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'nsmallest-per-partition'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">nsmallest</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'nsmallest'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">nsmallest</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="n">per_partition</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <span class="n">plot</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'plot'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.pop"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.pop">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">pop</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">item</span><span class="p">):</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="n">item</span><span class="p">]</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'popped'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">columns</span><span class="o">=</span><span class="p">[</span><span class="n">item</span><span class="p">]),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">return</span> <span class="n">result</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.quantile"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.quantile">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">quantile</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">q</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``quantile(axis="index")`` is not parallelizable. See</span> |
| <span class="sd"> `BEAM-12167 <https://issues.apache.org/jira/browse/BEAM-12167>`_ tracking</span> |
| <span class="sd"> the possible addition of an approximate, parallelizable implementation of</span> |
| <span class="sd"> quantile.</span> |
| |
| <span class="sd"> When using quantile with ``axis="columns"`` only a single ``q`` value can be</span> |
| <span class="sd"> specified."""</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">q</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"quantile(axis=columns) with multiple q values is not supported "</span> |
| <span class="s2">"because it transposes the input DataFrame. Note computing "</span> |
| <span class="s2">"an individual quantile across columns (e.g. "</span> |
| <span class="sa">f</span><span class="s2">"df.quantile(q=</span><span class="si">{</span><span class="n">q</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="si">!r}</span><span class="s2">, axis=</span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">) is supported."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> <span class="c1"># axis='index'</span> |
| <span class="c1"># TODO(BEAM-12167): Provide an option for approximate distributed</span> |
| <span class="c1"># quantiles</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"Computing quantiles across index cannot currently be parallelized. "</span> |
| <span class="s2">"See BEAM-12167 tracking the possible addition of an approximate, "</span> |
| <span class="s2">"parallelizable implementation of quantile."</span> |
| <span class="p">))</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'quantile'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">quantile</span><span class="p">(</span><span class="n">q</span><span class="o">=</span><span class="n">q</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.rename"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.rename">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">maybe_inplace</span> |
| <span class="k">def</span> <span class="nf">rename</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""rename is not parallelizable when ``axis="index"`` and</span> |
| <span class="sd"> ``errors="raise"``. It requires collecting all data on a single</span> |
| <span class="sd"> node in order to detect if one of the index values is missing."""</span> |
| <span class="n">rename_index</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s1">'index'</span> <span class="ow">in</span> <span class="n">kwargs</span> |
| <span class="ow">or</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'axis'</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">)</span> |
| <span class="ow">or</span> <span class="p">(</span><span class="s1">'columns'</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">kwargs</span> <span class="ow">and</span> <span class="s1">'axis'</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="p">))</span> |
| <span class="n">rename_columns</span> <span class="o">=</span> <span class="p">(</span> |
| <span class="s1">'columns'</span> <span class="ow">in</span> <span class="n">kwargs</span> |
| <span class="ow">or</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'axis'</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="n">rename_index</span><span class="p">:</span> |
| <span class="c1"># Technically, it's still partitioned by index, but it's no longer</span> |
| <span class="c1"># partitioned by the hash of the index.</span> |
| <span class="n">preserves_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">preserves_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'errors'</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> <span class="o">==</span> <span class="s1">'raise'</span> <span class="ow">and</span> <span class="n">rename_index</span><span class="p">:</span> |
| <span class="c1"># TODO: We could do this in parallel by creating a ConstantExpression</span> |
| <span class="c1"># with a series created from the mapper dict. Then Index() partitioning</span> |
| <span class="c1"># would co-locate the necessary index values and we could raise</span> |
| <span class="c1"># individually within each partition. Execution time errors are</span> |
| <span class="c1"># discouraged anyway so probably not worth the effort.</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"rename(errors='raise', axis='index') requires collecting all "</span> |
| <span class="s2">"data on a single node in order to detect missing index values."</span> |
| <span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">requires_partition_by</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| |
| <span class="n">proxy</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">if</span> <span class="n">rename_index</span><span class="p">:</span> |
| <span class="c1"># The proxy can't be computed by executing rename, it will error</span> |
| <span class="c1"># renaming the index.</span> |
| <span class="k">if</span> <span class="n">rename_columns</span><span class="p">:</span> |
| <span class="c1"># Note if both are being renamed, index and columns must be specified</span> |
| <span class="c1"># (not axis)</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="o">**</span><span class="p">{</span><span class="n">k</span><span class="p">:</span> <span class="n">v</span> <span class="k">for</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">)</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">k</span> <span class="o">==</span> <span class="s1">'index'</span><span class="p">})</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># No change in columns, reuse proxy</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'rename'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">preserves_partition_by</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires_partition_by</span><span class="p">))</span></div> |
| |
| <span class="n">rename_axis</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'rename_axis'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.round"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.round">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">round</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">decimals</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">decimals</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="c1"># Disallow passing a deferred Series in, our current partitioning model</span> |
| <span class="c1"># prevents us from using it correctly.</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">"Passing a deferred series to round() is not "</span> |
| <span class="s2">"supported, please use a concrete pd.Series "</span> |
| <span class="s2">"instance or a dictionary"</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'round'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">round</span><span class="p">(</span><span class="n">decimals</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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| <span class="p">)</span> |
| <span class="p">)</span></div> |
| |
| <span class="n">select_dtypes</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'select_dtypes'</span><span class="p">,</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.shift"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.shift">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">shift</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="n">freq</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""shift with ``axis="index" is only supported with ``freq`` specified and</span> |
| <span class="sd"> ``fill_value`` undefined. Other configurations make this operation</span> |
| <span class="sd"> order-sensitive."""</span> |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="n">preserves</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">freq</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="s1">'fill_value'</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="p">:</span> |
| <span class="n">fill_value</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'fill_value'</span><span class="p">,</span> <span class="s1">'NOT SET'</span><span class="p">)</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"shift(axis=</span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">) is only supported with freq defined, and "</span> |
| <span class="sa">f</span><span class="s2">"fill_value undefined (got freq=</span><span class="si">{</span><span class="n">freq</span><span class="si">!r}</span><span class="s2">,"</span> |
| <span class="sa">f</span><span class="s2">"fill_value=</span><span class="si">{</span><span class="n">fill_value</span><span class="si">!r}</span><span class="s2">). Other configurations are sensitive "</span> |
| <span class="s2">"to the order of the data because they require populating shifted "</span> |
| <span class="s2">"rows with `fill_value`."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="c1"># proxy generation fails in pandas <1.2</span> |
| <span class="c1"># Seems due to https://github.com/pandas-dev/pandas/issues/14811,</span> |
| <span class="c1"># bug with shift on empty indexes.</span> |
| <span class="c1"># Fortunately the proxy should be identical to the input.</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| |
| |
| <span class="c1"># index is modified, so no partitioning is preserved.</span> |
| <span class="n">preserves</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'shift'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">shift</span><span class="p">(</span><span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">freq</span><span class="o">=</span><span class="n">freq</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">preserves</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span></div> |
| |
| |
| <span class="n">shape</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'shape'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">))</span> |
| |
| <span class="n">stack</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="s1">'stack'</span><span class="p">,</span> |
| <span class="n">base</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">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">())</span> |
| |
| <span class="nb">all</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'all'</span><span class="p">)</span> |
| <span class="nb">any</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'any'</span><span class="p">)</span> |
| <span class="n">count</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'count'</span><span class="p">)</span> |
| <span class="n">describe</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'describe'</span><span class="p">)</span> |
| <span class="nb">max</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'max'</span><span class="p">)</span> |
| <span class="nb">min</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'min'</span><span class="p">)</span> |
| <span class="n">prod</span> <span class="o">=</span> <span class="n">product</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'prod'</span><span class="p">)</span> |
| <span class="nb">sum</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'sum'</span><span class="p">)</span> |
| <span class="n">mean</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'mean'</span><span class="p">)</span> |
| <span class="n">median</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'median'</span><span class="p">)</span> |
| <span class="n">nunique</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'nunique'</span><span class="p">)</span> |
| <span class="n">std</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'std'</span><span class="p">)</span> |
| <span class="n">var</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'var'</span><span class="p">)</span> |
| <span class="n">sem</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'sem'</span><span class="p">)</span> |
| <span class="n">mad</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'mad'</span><span class="p">)</span> |
| <span class="n">skew</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'skew'</span><span class="p">)</span> |
| <span class="n">kurt</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'kurt'</span><span class="p">)</span> |
| <span class="n">kurtosis</span> <span class="o">=</span> <span class="n">_agg_method</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="s1">'kurtosis'</span><span class="p">)</span> |
| |
| <span class="n">take</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'take'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'deprecated'</span><span class="p">)</span> |
| |
| <span class="n">to_records</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_records'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">to_dict</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_dict'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">to_numpy</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_numpy'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">to_string</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_string'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="n">to_sparse</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'to_sparse'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="n">transpose</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'transpose'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'non-deferred-columns'</span><span class="p">)</span> |
| <span class="n">T</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'T'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'non-deferred-columns'</span><span class="p">))</span> |
| |
| |
| <div class="viewcode-block" id="DeferredDataFrame.unstack"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.unstack">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">unstack</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="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""unstack cannot be used on :class:`DeferredDataFrame` instances with</span> |
| <span class="sd"> multiple index levels, because the columns in the output depend on the</span> |
| <span class="sd"> data."""</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'unstack'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">unstack</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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"unstack() is not supported on DataFrames with a multiple indexes, "</span> |
| <span class="s2">"because the columns in the output depend on the input data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span></div> |
| |
| <span class="n">update</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="s1">'update'</span><span class="p">,</span> |
| <span class="n">inplace</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">base</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">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| |
| <span class="n">values</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'values'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">))</span> |
| |
| <span class="n">style</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</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="s1">'style'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">))</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.melt"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.melt">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">melt</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">ignore_index</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``ignore_index=True`` is not supported, because it requires generating an</span> |
| <span class="sd"> order-sensitive index."""</span> |
| <span class="k">if</span> <span class="n">ignore_index</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"melt(ignore_index=True) is order sensitive because it requires "</span> |
| <span class="s2">"generating a new index based on the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'melt'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">melt</span><span class="p">(</span><span class="n">ignore_index</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span></div> |
| |
| <span class="k">if</span> <span class="nb">hasattr</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="s1">'value_counts'</span><span class="p">):</span> |
| <div class="viewcode-block" id="DeferredDataFrame.value_counts"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.value_counts">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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">def</span> <span class="nf">value_counts</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">subset</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">sort</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">normalize</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> |
| <span class="n">ascending</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">dropna</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="sd">"""``sort`` is ``False`` by default, and ``sort=True`` is not supported</span> |
| <span class="sd"> because it imposes an ordering on the dataset which likely will not be</span> |
| <span class="sd"> preserved."""</span> |
| |
| <span class="k">if</span> <span class="n">sort</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"value_counts(sort=True) is not supported because it imposes an "</span> |
| <span class="s2">"ordering on the dataset which likely will not be preserved."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="n">columns</span> <span class="o">=</span> <span class="n">subset</span> <span class="ow">or</span> <span class="nb">list</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">dropna</span><span class="p">:</span> |
| <span class="n">dropped</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">dropna</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">dropped</span> <span class="o">=</span> <span class="bp">self</span> |
| |
| <span class="n">result</span> <span class="o">=</span> <span class="n">dropped</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="n">dropna</span><span class="o">=</span><span class="n">dropna</span><span class="p">)</span><span class="o">.</span><span class="n">size</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="n">normalize</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">result</span><span class="o">/</span><span class="n">dropped</span><span class="o">.</span><span class="n">length</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">result</span></div> |
| |
| <span class="k">if</span> <span class="nb">hasattr</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="s1">'compare'</span><span class="p">):</span> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.compare"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.compare">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">compare</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">align_axis</span><span class="p">,</span> <span class="n">keep_shape</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""The default values ``align_axis=1 and ``keep_shape=False``</span> |
| <span class="sd"> are not supported, because the output columns depend on the data.</span> |
| <span class="sd"> To use ``align_axis=1``, please specify ``keep_shape=True``."""</span> |
| |
| <span class="n">preserve_partition</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="k">if</span> <span class="n">align_axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">keep_shape</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"compare(align_axis=</span><span class="si">{</span><span class="n">align_axis</span><span class="si">!r}</span><span class="s2">, keep_shape=</span><span class="si">{</span><span class="n">keep_shape</span><span class="si">!r}</span><span class="s2">) "</span> |
| <span class="s2">"is not allowed because the output columns depend on the data, "</span> |
| <span class="s2">"please specify keep_shape=True."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'non-deferred-columns'</span> |
| <span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">align_axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s1">'columns'</span><span class="p">):</span> |
| <span class="n">preserve_partition</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="k">elif</span> <span class="n">align_axis</span> <span class="ow">in</span> <span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s1">'index'</span><span class="p">):</span> |
| <span class="n">preserve_partition</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span> |
| <span class="s2">"align_axis must be one of ('index', 0, 'columns', 1). "</span> |
| <span class="sa">f</span><span class="s2">"got </span><span class="si">{</span><span class="n">align_axis</span><span class="si">!r}</span><span class="s2">."</span><span class="p">)</span> |
| |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'compare'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">compare</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">align_axis</span><span class="p">,</span> <span class="n">keep_shape</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">preserve_partition</span> |
| <span class="p">)</span> |
| <span class="p">)</span></div> |
| |
| <span class="k">def</span> <span class="nf">_idxmaxmin_helper</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">op</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">op</span> <span class="o">==</span> <span class="s1">'idxmax'</span><span class="p">:</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">idxmax</span> |
| <span class="k">elif</span> <span class="n">op</span> <span class="o">==</span> <span class="s1">'idxmin'</span><span class="p">:</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">idxmin</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">"op must be one of ('idxmax', 'idxmin'). "</span> |
| <span class="sa">f</span><span class="s2">"got </span><span class="si">{</span><span class="n">op</span><span class="si">!r}</span><span class="s2">."</span><span class="p">)</span> |
| |
| <span class="n">axis</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'axis'</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span> |
| |
| <span class="n">index_dtype</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">dtype</span> |
| <span class="n">columns_dtype</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">dtype</span> |
| |
| <span class="k">def</span> <span class="nf">compute_idx</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="n">indexes</span> <span class="o">=</span> <span class="n">func</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span><span class="o">.</span><span class="n">unique</span><span class="p">()</span> |
| <span class="k">if</span> <span class="n">pd</span><span class="o">.</span><span class="n">isna</span><span class="p">(</span><span class="n">indexes</span><span class="p">)</span><span class="o">.</span><span class="n">any</span><span class="p">():</span> |
| <span class="k">return</span> <span class="n">df</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">indexes</span><span class="p">]</span> |
| |
| <span class="k">if</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'index'</span><span class="p">,</span> <span class="mi">0</span><span class="p">):</span> |
| <span class="n">requires_partition</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()</span> |
| |
| <span class="n">proxy_index</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">([],</span> <span class="n">dtype</span><span class="o">=</span><span class="n">columns_dtype</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">Series</span><span class="p">([],</span> <span class="n">index</span><span class="o">=</span><span class="n">proxy_index</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="n">index_dtype</span><span class="p">)</span> |
| <span class="n">partition_proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| |
| <span class="n">idx_per_partition</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'idx-per-partition'</span><span class="p">,</span> |
| <span class="n">compute_idx</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">partition_proxy</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="p">)</span> |
| |
| <span class="k">elif</span> <span class="n">axis</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'columns'</span><span class="p">,</span> <span class="mi">1</span><span class="p">):</span> |
| <span class="n">requires_partition</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| |
| <span class="n">proxy_index</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">([],</span> <span class="n">dtype</span><span class="o">=</span><span class="n">index_dtype</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">Series</span><span class="p">([],</span> <span class="n">index</span><span class="o">=</span><span class="n">proxy_index</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="n">columns_dtype</span><span class="p">)</span> |
| |
| <span class="n">idx_per_partition</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">"axis must be one of ('index', 0, 'columns', 1). "</span> |
| <span class="sa">f</span><span class="s2">"got </span><span class="si">{</span><span class="n">axis</span><span class="si">!r}</span><span class="s2">."</span><span class="p">)</span> |
| |
| <span class="k">with</span> <span class="n">expressions</span><span class="o">.</span><span class="n">allow_non_parallel_operations</span><span class="p">(</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'idx'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">func</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="n">idx_per_partition</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires_partition</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()</span> |
| <span class="p">)</span> |
| <span class="p">)</span> |
| |
| |
| <div class="viewcode-block" id="DeferredDataFrame.idxmin"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.idxmin">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">idxmin</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_idxmaxmin_helper</span><span class="p">(</span><span class="s1">'idxmin'</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="DeferredDataFrame.idxmax"><a class="viewcode-back" href="../../../apache_beam.dataframe.frames.html#apache_beam.dataframe.frames.DeferredDataFrame.idxmax">[docs]</a> <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</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="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</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="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</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">def</span> <span class="nf">idxmax</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_idxmaxmin_helper</span><span class="p">(</span><span class="s1">'idxmax'</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div></div> |
| |
| |
| <span class="k">for</span> <span class="n">io_func</span> <span class="ow">in</span> <span class="nb">dir</span><span class="p">(</span><span class="n">io</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">io_func</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">'to_'</span><span class="p">):</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredDataFrame</span><span class="p">,</span> <span class="n">io_func</span><span class="p">,</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">io</span><span class="p">,</span> <span class="n">io_func</span><span class="p">))</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredSeries</span><span class="p">,</span> <span class="n">io_func</span><span class="p">,</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">io</span><span class="p">,</span> <span class="n">io_func</span><span class="p">))</span> |
| |
| |
| <span class="k">for</span> <span class="n">meth</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'filter'</span><span class="p">,</span> <span class="p">):</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredDataFrame</span><span class="p">,</span> <span class="n">meth</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">meth</span><span class="p">,</span> <span class="n">base</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="nd">@populate_not_implemented</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">DeferredGroupBy</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</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">expr</span><span class="p">,</span> <span class="n">kwargs</span><span class="p">,</span> |
| <span class="n">ungrouped</span><span class="p">:</span> <span class="n">expressions</span><span class="o">.</span><span class="n">Expression</span><span class="p">[</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">],</span> |
| <span class="n">ungrouped_with_index</span><span class="p">:</span> <span class="n">expressions</span><span class="o">.</span><span class="n">Expression</span><span class="p">[</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">],</span> <span class="c1"># pylint: disable=line-too-long</span> |
| <span class="n">grouping_columns</span><span class="p">,</span> |
| <span class="n">grouping_indexes</span><span class="p">,</span> |
| <span class="n">projection</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="sd">"""This object represents the result of::</span> |
| |
| <span class="sd"> ungrouped.groupby(level=[grouping_indexes + grouping_columns],</span> |
| <span class="sd"> **kwargs)[projection]</span> |
| |
| <span class="sd"> :param expr: An expression to compute a pandas GroupBy object. Convenient</span> |
| <span class="sd"> for unliftable aggregations.</span> |
| <span class="sd"> :param ungrouped: An expression to compute the DataFrame pre-grouping, the</span> |
| <span class="sd"> (Multi)Index contains only the grouping columns/indexes.</span> |
| <span class="sd"> :param ungrouped_with_index: Same as ungrouped, except the index includes</span> |
| <span class="sd"> all of the original indexes as well as any grouping columns. This is</span> |
| <span class="sd"> important for operations that expose the original index, e.g. .apply(),</span> |
| <span class="sd"> but we only use it when necessary to avoid unnessary data transfer and</span> |
| <span class="sd"> GBKs.</span> |
| <span class="sd"> :param grouping_columns: list of column labels that were in the original</span> |
| <span class="sd"> groupby(..) ``by`` parameter. Only relevant for grouped DataFrames.</span> |
| <span class="sd"> :param grouping_indexes: list of index names (or index level numbers) to be</span> |
| <span class="sd"> grouped.</span> |
| <span class="sd"> :param kwargs: Keywords args passed to the original groupby(..) call."""</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">expr</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span> <span class="o">=</span> <span class="n">ungrouped</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped_with_index</span> <span class="o">=</span> <span class="n">ungrouped_with_index</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_projection</span> <span class="o">=</span> <span class="n">projection</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span> <span class="o">=</span> <span class="n">grouping_columns</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_indexes</span> <span class="o">=</span> <span class="n">grouping_indexes</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_kwargs</span> <span class="o">=</span> <span class="n">kwargs</span> |
| |
| <span class="k">if</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'dropna'</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span> <span class="ow">is</span> <span class="kc">False</span> <span class="ow">and</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span> <span class="o">></span> <span class="mi">1</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="s2">"dropna=False does not work as intended in the Beam DataFrame API "</span> |
| <span class="s2">"when grouping on multiple columns or indexes (See BEAM-12495)."</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="fm">__getattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">DeferredGroupBy</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'groupby_project'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">gb</span><span class="p">:</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">gb</span><span class="p">,</span> <span class="n">name</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()),</span> |
| <span class="bp">self</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">_ungrouped</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped_with_index</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_indexes</span><span class="p">,</span> |
| <span class="n">projection</span><span class="o">=</span><span class="n">name</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="fm">__getitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">DeferredGroupBy</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'groupby_project'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">gb</span><span class="p">:</span> <span class="n">gb</span><span class="p">[</span><span class="n">name</span><span class="p">],</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()),</span> |
| <span class="bp">self</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">_ungrouped</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped_with_index</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_indexes</span><span class="p">,</span> |
| <span class="n">projection</span><span class="o">=</span><span class="n">name</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">agg</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">fn</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">_is_associative</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_liftable_agg</span><span class="p">(</span><span class="n">fn</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="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">_is_liftable_with_sum</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_liftable_agg</span><span class="p">(</span><span class="n">fn</span><span class="p">,</span> <span class="n">postagg_meth</span><span class="o">=</span><span class="s1">'sum'</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="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">_is_unliftable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_unliftable_agg</span><span class="p">(</span><span class="n">fn</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="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">DeferredDataFrame</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'agg'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">gb</span><span class="p">:</span> <span class="n">gb</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">fn</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="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</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="sa">f</span><span class="s2">"GroupBy.agg(func=</span><span class="si">{</span><span class="n">fn</span><span class="si">!r}</span><span class="s2">)"</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">ndim</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">ndim</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">apply</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</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="sd">"""Note that ``func`` will be called once during pipeline construction time</span> |
| <span class="sd"> with an empty pandas object, so take care if ``func`` has a side effect.</span> |
| |
| <span class="sd"> When called with an empty pandas object, ``func`` is expected to return an</span> |
| <span class="sd"> object of the same type as what will be returned when the pipeline is</span> |
| <span class="sd"> processing actual data. If the result is a pandas object it should have the</span> |
| <span class="sd"> same type and name (for a Series) or column types and names (for</span> |
| <span class="sd"> a DataFrame) as the actual results.</span> |
| |
| <span class="sd"> Note that in pandas, ``apply`` attempts to detect if the index is unmodified</span> |
| <span class="sd"> in ``func`` (indicating ``func`` is a transform) and drops the duplicate</span> |
| <span class="sd"> index in the output. To determine this, pandas tests the indexes for</span> |
| <span class="sd"> equality. However, Beam cannot do this since it is sensitive to the input</span> |
| <span class="sd"> data; instead this implementation tests if the indexes are equivalent</span> |
| <span class="sd"> with ``is``. See the `pandas 1.4.0 release notes</span> |
| <span class="sd"> <https://pandas.pydata.org/docs/dev/whatsnew/v1.4.0.html#groupby-apply-consistent-transform-detection>`_</span> |
| <span class="sd"> for a good explanation of the distinction between these approaches. In</span> |
| <span class="sd"> practice, this just means that in some cases the Beam result will have</span> |
| <span class="sd"> a duplicate index, whereas pandas would have dropped it."""</span> |
| |
| <span class="n">project</span> <span class="o">=</span> <span class="n">_maybe_project_func</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_projection</span><span class="p">)</span> |
| <span class="n">grouping_indexes</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_indexes</span> |
| <span class="n">grouping_columns</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span> |
| |
| <span class="c1"># Unfortunately pandas does not execute func to determine the right proxy.</span> |
| <span class="c1"># We run user func on a proxy here to detect the return type and generate</span> |
| <span class="c1"># the proxy.</span> |
| <span class="n">fn_input</span> <span class="o">=</span> <span class="n">project</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped_with_index</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span> |
| <span class="n">grouping_columns</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">))</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">func</span><span class="p">(</span><span class="n">fn_input</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">index_to_arrays</span><span class="p">(</span><span class="n">index</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">[</span><span class="n">index</span><span class="o">.</span><span class="n">get_level_values</span><span class="p">(</span><span class="n">level</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">level</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">)]</span> |
| |
| |
| <span class="c1"># By default do_apply will just call pandas apply()</span> |
| <span class="c1"># We override it below if necessary</span> |
| <span class="n">do_apply</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">gb</span><span class="p">:</span> <span class="n">gb</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">result</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">)</span> <span class="ow">and</span> |
| <span class="n">result</span><span class="o">.</span><span class="n">index</span> <span class="ow">is</span> <span class="n">fn_input</span><span class="o">.</span><span class="n">index</span><span class="p">):</span> |
| <span class="c1"># Special case where apply fn is a transform</span> |
| <span class="c1"># Note we trust that if the user fn produces a proxy with the identical</span> |
| <span class="c1"># index, it will produce results with identical indexes at execution</span> |
| <span class="c1"># time too</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="n">result</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">result</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="c1"># apply fn is not a transform, we need to make sure the original index</span> |
| <span class="c1"># values are prepended to the result's index</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="n">result</span><span class="p">[:</span><span class="mi">0</span><span class="p">]</span> |
| |
| <span class="c1"># First adjust proxy</span> |
| <span class="n">proxy</span><span class="o">.</span><span class="n">index</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="o">.</span><span class="n">from_arrays</span><span class="p">(</span> |
| <span class="n">index_to_arrays</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="p">)</span> <span class="o">+</span> |
| <span class="n">index_to_arrays</span><span class="p">(</span><span class="n">proxy</span><span class="o">.</span><span class="n">index</span><span class="p">),</span> |
| <span class="n">names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span> <span class="o">+</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="c1"># Then override do_apply function</span> |
| <span class="n">new_index_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">new_index_names</span><span class="p">)</span> <span class="o">></span> <span class="mi">1</span><span class="p">:</span> |
| <span class="k">def</span> <span class="nf">add_key_index</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">df</span><span class="p">):</span> |
| <span class="c1"># df is a dataframe or Series representing the result of func for</span> |
| <span class="c1"># a single key</span> |
| <span class="c1"># key is a tuple with the MultiIndex values for this key</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">index</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="o">.</span><span class="n">from_arrays</span><span class="p">(</span> |
| <span class="p">[[</span><span class="n">key</span><span class="p">[</span><span class="n">i</span><span class="p">]]</span> <span class="o">*</span> <span class="nb">len</span><span class="p">(</span><span class="n">df</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">new_index_names</span><span class="p">))]</span> <span class="o">+</span> |
| <span class="n">index_to_arrays</span><span class="p">(</span><span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="p">),</span> |
| <span class="n">names</span><span class="o">=</span><span class="n">new_index_names</span> <span class="o">+</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="k">return</span> <span class="n">df</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">def</span> <span class="nf">add_key_index</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">df</span><span class="p">):</span> |
| <span class="c1"># df is a dataframe or Series representing the result of func for</span> |
| <span class="c1"># a single key</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">index</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="o">.</span><span class="n">from_arrays</span><span class="p">(</span> |
| <span class="p">[[</span><span class="n">key</span><span class="p">]</span> <span class="o">*</span> <span class="nb">len</span><span class="p">(</span><span class="n">df</span><span class="p">)]</span> <span class="o">+</span> <span class="n">index_to_arrays</span><span class="p">(</span><span class="n">df</span><span class="o">.</span><span class="n">index</span><span class="p">),</span> |
| <span class="n">names</span><span class="o">=</span><span class="n">new_index_names</span> <span class="o">+</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="k">return</span> <span class="n">df</span> |
| |
| |
| <span class="n">do_apply</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">gb</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">concat</span><span class="p">([</span> |
| <span class="n">add_key_index</span><span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">func</span><span class="p">(</span><span class="n">gb</span><span class="o">.</span><span class="n">get_group</span><span class="p">(</span><span class="n">k</span><span class="p">),</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">))</span> |
| <span class="k">for</span> <span class="n">k</span> <span class="ow">in</span> <span class="n">gb</span><span class="o">.</span><span class="n">groups</span><span class="o">.</span><span class="n">keys</span><span class="p">()])</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">result</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">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">fn_input</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="c1"># DataFrameGroupBy</span> |
| <span class="c1"># In this case pandas transposes the Series result, s.t. the Series</span> |
| <span class="c1"># index values are the columns, and the grouping keys are the new index</span> |
| <span class="c1"># values.</span> |
| <span class="n">dtype</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">([</span><span class="n">result</span><span class="p">])</span><span class="o">.</span><span class="n">dtype</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="n">result</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> |
| <span class="n">dtype</span><span class="o">=</span><span class="n">result</span><span class="o">.</span><span class="n">dtype</span><span class="p">,</span> |
| <span class="n">index</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">fn_input</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"># SeriesGroupBy</span> |
| <span class="c1"># In this case the output is still a Series, but with an additional</span> |
| <span class="c1"># index with the grouping keys.</span> |
| <span class="n">proxy</span> <span class="o">=</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">result</span><span class="o">.</span><span class="n">dtype</span><span class="p">,</span> |
| <span class="n">name</span><span class="o">=</span><span class="n">result</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> |
| <span class="n">index</span><span class="o">=</span><span class="n">index_to_arrays</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="p">)</span> <span class="o">+</span> |
| <span class="n">index_to_arrays</span><span class="p">(</span><span class="n">result</span><span class="p">[:</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">index</span><span class="p">))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># The user fn returns some non-pandas type. The expected result is a</span> |
| <span class="c1"># Series where each element is the result of one user fn call.</span> |
| <span class="n">dtype</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">([</span><span class="n">result</span><span class="p">])</span><span class="o">.</span><span class="n">dtype</span> |
| <span class="n">proxy</span> <span class="o">=</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="n">index</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">do_partition_apply</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="c1"># Remove columns from index, we only needed them there for partitioning</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">grouping_columns</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| |
| <span class="n">gb</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">level</span><span class="o">=</span><span class="n">grouping_indexes</span> <span class="ow">or</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="n">by</span><span class="o">=</span><span class="n">grouping_columns</span> <span class="ow">or</span> <span class="kc">None</span><span class="p">)</span> |
| |
| <span class="n">gb</span> <span class="o">=</span> <span class="n">project</span><span class="p">(</span><span class="n">gb</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">do_apply</span><span class="p">(</span><span class="n">gb</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">DeferredDataFrame</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'apply'</span><span class="p">,</span> |
| <span class="n">do_partition_apply</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped_with_index</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span><span class="n">grouping_indexes</span> <span class="o">+</span> |
| <span class="n">grouping_columns</span><span class="p">),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span><span class="n">grouping_indexes</span><span class="p">)))</span> |
| |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">transform</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">fn</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="sd">"""Note that ``func`` will be called once during pipeline construction time</span> |
| <span class="sd"> with an empty pandas object, so take care if ``func`` has a side effect.</span> |
| |
| <span class="sd"> When called with an empty pandas object, ``func`` is expected to return an</span> |
| <span class="sd"> object of the same type as what will be returned when the pipeline is</span> |
| <span class="sd"> processing actual data. The result should have the same type and name (for</span> |
| <span class="sd"> a Series) or column types and names (for a DataFrame) as the actual</span> |
| <span class="sd"> results."""</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">callable</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span> |
| <span class="s2">"String functions are not yet supported in transform."</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span> <span class="ow">and</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">_projection</span><span class="p">:</span> |
| <span class="n">grouping_columns</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span> |
| <span class="k">def</span> <span class="nf">fn_wrapper</span><span class="p">(</span><span class="n">x</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="n">x</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="n">grouping_columns</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">fn</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">fn_wrapper</span> <span class="o">=</span> <span class="n">fn</span> |
| |
| <span class="n">project</span> <span class="o">=</span> <span class="n">_maybe_project_func</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_projection</span><span class="p">)</span> |
| |
| <span class="c1"># pandas cannot execute fn to determine the right proxy.</span> |
| <span class="c1"># We run user fn on a proxy here to detect the return type and generate the</span> |
| <span class="c1"># proxy.</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">fn_wrapper</span><span class="p">(</span><span class="n">project</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped_with_index</span><span class="o">.</span><span class="n">proxy</span><span class="p">()))</span> |
| <span class="n">parent_frame</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">args</span><span class="p">()[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">result</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">generic</span><span class="o">.</span><span class="n">NDFrame</span><span class="p">):</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="n">result</span><span class="p">[:</span><span class="mi">0</span><span class="p">]</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># The user fn returns some non-pandas type. The expected result is a</span> |
| <span class="c1"># Series where each element is the result of one user fn call.</span> |
| <span class="n">dtype</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">([</span><span class="n">result</span><span class="p">])</span><span class="o">.</span><span class="n">dtype</span> |
| <span class="n">proxy</span> <span class="o">=</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="n">name</span><span class="o">=</span><span class="n">project</span><span class="p">(</span><span class="n">parent_frame</span><span class="p">)</span><span class="o">.</span><span class="n">name</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_projection</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="n">proxy</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_projection</span> |
| |
| <span class="c1"># The final result will have the original indexes</span> |
| <span class="n">proxy</span><span class="o">.</span><span class="n">index</span> <span class="o">=</span> <span class="n">parent_frame</span><span class="o">.</span><span class="n">index</span> |
| |
| <span class="n">levels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_indexes</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span> |
| |
| <span class="k">return</span> <span class="n">DeferredDataFrame</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'transform'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">project</span><span class="p">(</span><span class="n">df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">level</span><span class="o">=</span><span class="n">levels</span><span class="p">))</span><span class="o">.</span><span class="n">transform</span><span class="p">(</span> |
| <span class="n">fn_wrapper</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="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped_with_index</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span><span class="n">levels</span><span class="p">),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_grouping_indexes</span><span class="p">)))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">pipe</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span> |
| <span class="n">func</span><span class="p">,</span> <span class="n">data</span> <span class="o">=</span> <span class="n">func</span> |
| <span class="n">kwargs</span><span class="p">[</span><span class="n">data</span><span class="p">]</span> <span class="o">=</span> <span class="bp">self</span> |
| <span class="k">return</span> <span class="n">func</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">func</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="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">filter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">dropna</span><span class="o">=</span><span class="kc">True</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">func</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="ow">not</span> <span class="n">callable</span><span class="p">(</span><span class="n">func</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">"func must be specified and it must be callable"</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">apply_fn</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">func</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">df</span> |
| <span class="k">elif</span> <span class="ow">not</span> <span class="n">dropna</span><span class="p">:</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="n">result</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:,</span> <span class="p">:]</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">nan</span> |
| <span class="k">return</span> <span class="n">result</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:</span><span class="mi">0</span><span class="p">]</span> |
| |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">apply_fn</span><span class="p">)</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_grouping_columns</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">dtypes</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'dtypes'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">gb</span><span class="p">:</span> <span class="n">gb</span><span class="o">.</span><span class="n">dtypes</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()</span> |
| <span class="p">)</span> |
| <span class="p">)</span> |
| |
| <span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'value_counts'</span><span class="p">):</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">value_counts</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""</span> |
| <span class="sd"> DataFrameGroupBy.value_counts() is the same as DataFrame.value_counts()</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'value_counts'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">value_counts</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="p">)</span> |
| |
| <span class="n">fillna</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'fillna'</span><span class="p">,</span> <span class="n">explanation</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"df.fillna() should be used instead. Only method=None is supported "</span> |
| <span class="s2">"because other methods are order-sensitive. df.groupby(..).fillna() "</span> |
| <span class="s2">"without a method is equivalent to df.fillna()."</span><span class="p">))</span> |
| |
| <span class="n">ffill</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'ffill'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="n">bfill</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'bfill'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="n">pad</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'pad'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="n">backfill</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'backfill'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="n">aggregate</span> <span class="o">=</span> <span class="n">agg</span> |
| |
| <span class="n">hist</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'hist'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| <span class="n">plot</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'plot'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| <span class="n">boxplot</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'boxplot'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| |
| <span class="n">head</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'head'</span><span class="p">,</span> <span class="n">explanation</span><span class="o">=</span><span class="n">_PEEK_METHOD_EXPLANATION</span><span class="p">)</span> |
| <span class="n">tail</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'tail'</span><span class="p">,</span> <span class="n">explanation</span><span class="o">=</span><span class="n">_PEEK_METHOD_EXPLANATION</span><span class="p">)</span> |
| |
| <span class="n">first</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">not_implemented_method</span><span class="p">(</span><span class="s1">'first'</span><span class="p">,</span> <span class="n">base_type</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">last</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">not_implemented_method</span><span class="p">(</span><span class="s1">'last'</span><span class="p">,</span> <span class="n">base_type</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">nth</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'nth'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">))</span> |
| <span class="n">cumcount</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'cumcount'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cummax</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'cummax'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cummin</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'cummin'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cumsum</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'cumsum'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">cumprod</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'cumprod'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">diff</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'diff'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">shift</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'shift'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| |
| <span class="n">pct_change</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'pct_change'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| <span class="n">ohlc</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'ohlc'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s1">'order-sensitive'</span><span class="p">)</span> |
| |
| <span class="c1"># TODO(BEAM-12169): Consider allowing this for categorical keys.</span> |
| <span class="fm">__len__</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'__len__'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">groups</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'groups'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">))</span> |
| <span class="n">indices</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'indices'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">))</span> |
| |
| <span class="n">resample</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'resample'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'event-time-semantics'</span><span class="p">)</span> |
| <span class="n">rolling</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'rolling'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'event-time-semantics'</span><span class="p">)</span> |
| <span class="n">ewm</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'ewm'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"event-time-semantics"</span><span class="p">)</span> |
| <span class="n">expanding</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'expanding'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"event-time-semantics"</span><span class="p">)</span> |
| |
| <span class="n">tshift</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'tshift'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"deprecated"</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_maybe_project_func</span><span class="p">(</span><span class="n">projection</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]):</span> |
| <span class="sd">""" Returns identity func if projection is empty or None, else returns</span> |
| <span class="sd"> a function that projects the specified columns. """</span> |
| <span class="k">if</span> <span class="n">projection</span><span class="p">:</span> |
| <span class="k">return</span> <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="p">[</span><span class="n">projection</span><span class="p">]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_liftable_agg</span><span class="p">(</span><span class="n">meth</span><span class="p">,</span> <span class="n">postagg_meth</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span> |
| <span class="n">agg_name</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">name_and_func</span><span class="p">(</span><span class="n">meth</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="n">postagg_meth</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">post_agg_name</span> <span class="o">=</span> <span class="n">agg_name</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">post_agg_name</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">name_and_func</span><span class="p">(</span><span class="n">postagg_meth</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="n">name</span><span class="o">=</span><span class="n">agg_name</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">wrapper</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="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">DeferredGroupBy</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="s1">'min_count'</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">_unliftable_agg</span><span class="p">(</span><span class="n">meth</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="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="n">to_group</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span> |
| <span class="n">is_categorical_grouping</span> <span class="o">=</span> <span class="nb">any</span><span class="p">(</span><span class="n">to_group</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">is_categorical</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_indexes</span><span class="p">)</span> |
| <span class="n">groupby_kwargs</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_kwargs</span> |
| |
| <span class="c1"># Don't include un-observed categorical values in the preagg</span> |
| <span class="n">preagg_groupby_kwargs</span> <span class="o">=</span> <span class="n">groupby_kwargs</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="n">preagg_groupby_kwargs</span><span class="p">[</span><span class="s1">'observed'</span><span class="p">]</span> <span class="o">=</span> <span class="kc">True</span> |
| |
| <span class="n">project</span> <span class="o">=</span> <span class="n">_maybe_project_func</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_projection</span><span class="p">)</span> |
| <span class="n">pre_agg</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'pre_combine_'</span> <span class="o">+</span> <span class="n">agg_name</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="nb">getattr</span><span class="p">(</span> |
| <span class="n">project</span><span class="p">(</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">level</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="nb">range</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">nlevels</span><span class="p">)),</span> |
| <span class="o">**</span><span class="n">preagg_groupby_kwargs</span><span class="p">)</span> |
| <span class="p">),</span> |
| <span class="n">agg_name</span><span class="p">)(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| |
| |
| <span class="n">post_agg</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'post_combine_'</span> <span class="o">+</span> <span class="n">post_agg_name</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="nb">getattr</span><span class="p">(</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">level</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="nb">range</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">nlevels</span><span class="p">)),</span> |
| <span class="o">**</span><span class="n">groupby_kwargs</span><span class="p">),</span> |
| <span class="n">post_agg_name</span><span class="p">)(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="n">pre_agg</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="p">(</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"Aggregations grouped by a categorical column are not currently "</span> |
| <span class="s2">"parallelizable (BEAM-11190)."</span> |
| <span class="p">))</span> |
| <span class="k">if</span> <span class="n">is_categorical_grouping</span> |
| <span class="k">else</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span><span class="n">post_agg</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">wrapper</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_unliftable_agg</span><span class="p">(</span><span class="n">meth</span><span class="p">):</span> |
| <span class="n">agg_name</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">name_and_func</span><span class="p">(</span><span class="n">meth</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="n">name</span><span class="o">=</span><span class="n">agg_name</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">wrapper</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="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">DeferredGroupBy</span><span class="p">)</span> |
| |
| <span class="n">to_group</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span> |
| <span class="n">is_categorical_grouping</span> <span class="o">=</span> <span class="nb">any</span><span class="p">(</span><span class="n">to_group</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">is_categorical</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_grouping_indexes</span><span class="p">)</span> |
| |
| <span class="n">groupby_kwargs</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_kwargs</span> |
| <span class="n">project</span> <span class="o">=</span> <span class="n">_maybe_project_func</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_projection</span><span class="p">)</span> |
| <span class="n">post_agg</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="n">agg_name</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">project</span><span class="p">(</span> |
| <span class="n">df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">level</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="nb">range</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">nlevels</span><span class="p">)),</span> |
| <span class="o">**</span><span class="n">groupby_kwargs</span><span class="p">),</span> |
| <span class="p">),</span> <span class="n">agg_name</span><span class="p">)(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="p">(</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"Aggregations grouped by a categorical column are not currently "</span> |
| <span class="s2">"parallelizable (BEAM-11190)."</span> |
| <span class="p">))</span> |
| <span class="k">if</span> <span class="n">is_categorical_grouping</span> |
| <span class="k">else</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()),</span> |
| <span class="c1"># Some aggregation methods (e.g. corr/cov) add additional index levels.</span> |
| <span class="c1"># We only preserve the ones that existed _before_ the groupby.</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span> |
| <span class="nb">list</span><span class="p">(</span><span class="nb">range</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_ungrouped</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">))))</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span><span class="n">post_agg</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">wrapper</span> |
| |
| <span class="k">for</span> <span class="n">meth</span> <span class="ow">in</span> <span class="n">LIFTABLE_AGGREGATIONS</span><span class="p">:</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredGroupBy</span><span class="p">,</span> <span class="n">meth</span><span class="p">,</span> <span class="n">_liftable_agg</span><span class="p">(</span><span class="n">meth</span><span class="p">))</span> |
| <span class="k">for</span> <span class="n">meth</span> <span class="ow">in</span> <span class="n">LIFTABLE_WITH_SUM_AGGREGATIONS</span><span class="p">:</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredGroupBy</span><span class="p">,</span> <span class="n">meth</span><span class="p">,</span> <span class="n">_liftable_agg</span><span class="p">(</span><span class="n">meth</span><span class="p">,</span> <span class="n">postagg_meth</span><span class="o">=</span><span class="s1">'sum'</span><span class="p">))</span> |
| <span class="k">for</span> <span class="n">meth</span> <span class="ow">in</span> <span class="n">UNLIFTABLE_AGGREGATIONS</span><span class="p">:</span> |
| <span class="k">if</span> <span class="n">meth</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'kurt'</span><span class="p">,</span> <span class="s1">'kurtosis'</span><span class="p">):</span> |
| <span class="c1"># pandas doesn't currently allow kurtosis on GroupBy:</span> |
| <span class="c1"># https://github.com/pandas-dev/pandas/issues/40139</span> |
| <span class="k">continue</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredGroupBy</span><span class="p">,</span> <span class="n">meth</span><span class="p">,</span> <span class="n">_unliftable_agg</span><span class="p">(</span><span class="n">meth</span><span class="p">))</span> |
| |
| <span class="k">def</span> <span class="nf">_check_str_or_np_builtin</span><span class="p">(</span><span class="n">agg_func</span><span class="p">,</span> <span class="n">func_list</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">agg_func</span> <span class="ow">in</span> <span class="n">func_list</span> <span class="ow">or</span> <span class="p">(</span> |
| <span class="nb">getattr</span><span class="p">(</span><span class="n">agg_func</span><span class="p">,</span> <span class="s1">'__name__'</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span> <span class="ow">in</span> <span class="n">func_list</span> |
| <span class="ow">and</span> <span class="n">agg_func</span><span class="o">.</span><span class="vm">__module__</span> <span class="ow">in</span> <span class="p">(</span><span class="s1">'numpy'</span><span class="p">,</span> <span class="s1">'builtins'</span><span class="p">))</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_is_associative</span><span class="p">(</span><span class="n">agg_func</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_check_str_or_np_builtin</span><span class="p">(</span><span class="n">agg_func</span><span class="p">,</span> <span class="n">LIFTABLE_AGGREGATIONS</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_is_liftable_with_sum</span><span class="p">(</span><span class="n">agg_func</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_check_str_or_np_builtin</span><span class="p">(</span><span class="n">agg_func</span><span class="p">,</span> <span class="n">LIFTABLE_WITH_SUM_AGGREGATIONS</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_is_unliftable</span><span class="p">(</span><span class="n">agg_func</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_check_str_or_np_builtin</span><span class="p">(</span><span class="n">agg_func</span><span class="p">,</span> <span class="n">UNLIFTABLE_AGGREGATIONS</span><span class="p">)</span> |
| |
| <span class="n">NUMERIC_AGGREGATIONS</span> <span class="o">=</span> <span class="p">[</span><span class="s1">'max'</span><span class="p">,</span> <span class="s1">'min'</span><span class="p">,</span> <span class="s1">'prod'</span><span class="p">,</span> <span class="s1">'sum'</span><span class="p">,</span> <span class="s1">'mean'</span><span class="p">,</span> <span class="s1">'median'</span><span class="p">,</span> <span class="s1">'std'</span><span class="p">,</span> |
| <span class="s1">'var'</span><span class="p">,</span> <span class="s1">'sem'</span><span class="p">,</span> <span class="s1">'mad'</span><span class="p">,</span> <span class="s1">'skew'</span><span class="p">,</span> <span class="s1">'kurt'</span><span class="p">,</span> <span class="s1">'kurtosis'</span><span class="p">]</span> |
| |
| <span class="k">def</span> <span class="nf">_is_numeric</span><span class="p">(</span><span class="n">agg_func</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_check_str_or_np_builtin</span><span class="p">(</span><span class="n">agg_func</span><span class="p">,</span> <span class="n">NUMERIC_AGGREGATIONS</span><span class="p">)</span> |
| |
| |
| <span class="nd">@populate_not_implemented</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">_DeferredGroupByCols</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="c1"># It's not clear that all of these make sense in Pandas either...</span> |
| <span class="n">agg</span> <span class="o">=</span> <span class="n">aggregate</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'agg'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="nb">any</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'any'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="nb">all</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'all'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">boxplot</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'boxplot'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| <span class="n">describe</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">not_implemented_method</span><span class="p">(</span><span class="s1">'describe'</span><span class="p">,</span> |
| <span class="n">base_type</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">diff</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'diff'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">fillna</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'fillna'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="nb">filter</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'filter'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">first</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'first'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">get_group</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'get_group'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">head</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'head'</span><span class="p">,</span> <span class="n">explanation</span><span class="o">=</span><span class="n">_PEEK_METHOD_EXPLANATION</span><span class="p">)</span> |
| <span class="n">hist</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'hist'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| <span class="n">idxmax</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'idxmax'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">idxmin</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'idxmin'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">last</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'last'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">mad</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'mad'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="nb">max</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'max'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">mean</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'mean'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">median</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'median'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="nb">min</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'min'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">nunique</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'nunique'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">plot</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'plot'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"plotting-tools"</span><span class="p">)</span> |
| <span class="n">prod</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'prod'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">quantile</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'quantile'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">shift</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'shift'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">size</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'size'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">skew</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'skew'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">std</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'std'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="nb">sum</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'sum'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">tail</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'tail'</span><span class="p">,</span> <span class="n">explanation</span><span class="o">=</span><span class="n">_PEEK_METHOD_EXPLANATION</span><span class="p">)</span> |
| <span class="n">take</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">DataFrameGroupBy</span><span class="p">,</span> <span class="s1">'take'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s1">'deprecated'</span><span class="p">)</span> |
| <span class="n">tshift</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'tshift'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="n">var</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="s1">'var'</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">groups</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">groups</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">indices</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">indices</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">ndim</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">ndim</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">DataFrameGroupBy</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">ngroups</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">ngroups</span> |
| |
| |
| <span class="nd">@populate_not_implemented</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">base</span><span class="o">.</span><span class="n">Index</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">_DeferredIndex</span><span class="p">(</span><span class="nb">object</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">frame</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span> <span class="o">=</span> <span class="n">frame</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">names</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span> |
| |
| <span class="nd">@names</span><span class="o">.</span><span class="n">setter</span> |
| <span class="k">def</span> <span class="nf">names</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">set_index_names</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">copy</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="o">=</span> <span class="n">value</span> |
| <span class="k">return</span> <span class="n">df</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'set_index_names'</span><span class="p">,</span> |
| <span class="n">set_index_names</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">())</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">name</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span> |
| |
| <span class="nd">@name</span><span class="o">.</span><span class="n">setter</span> |
| <span class="k">def</span> <span class="nf">name</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">names</span> <span class="o">=</span> <span class="p">[</span><span class="n">value</span><span class="p">]</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">ndim</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">ndim</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">dtype</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">dtype</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">nlevels</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span> |
| |
| <span class="k">def</span> <span class="fm">__getattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">name</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">'index.</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="n">name</span><span class="p">)</span> |
| |
| |
| <span class="nd">@populate_not_implemented</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexing</span><span class="o">.</span><span class="n">_LocIndexer</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">_DeferredLoc</span><span class="p">(</span><span class="nb">object</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">frame</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span> <span class="o">=</span> <span class="n">frame</span> |
| |
| <span class="k">def</span> <span class="fm">__getitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span> |
| <span class="n">rows</span><span class="p">,</span> <span class="n">cols</span> <span class="o">=</span> <span class="n">key</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">rows</span><span class="p">][</span><span class="n">cols</span><span class="p">]</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> <span class="n">key</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="nb">bool</span><span class="p">):</span> |
| <span class="c1"># Aligned by numerical key.</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">key</span><span class="p">))</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="c1"># Select rows, but behaves poorly on missing values.</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">key</span><span class="p">))</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">slice</span><span class="p">):</span> |
| <span class="n">args</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="p">]</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">df</span><span class="p">,</span> <span class="n">key</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> |
| <span class="k">if</span> <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">dtypes</span><span class="o">.</span><span class="n">common</span><span class="o">.</span><span class="n">is_bool_dtype</span><span class="p">(</span><span class="n">key</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()):</span> |
| <span class="c1"># Boolean indexer, just pass it in as-is</span> |
| <span class="n">args</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">key</span><span class="o">.</span><span class="n">_expr</span><span class="p">]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Likely a DeferredSeries of labels, overwrite the key's index with it's</span> |
| <span class="c1"># values so we can colocate them with the labels they're selecting</span> |
| <span class="k">def</span> <span class="nf">data_to_index</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="n">s</span> <span class="o">=</span> <span class="n">s</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> |
| <span class="n">s</span><span class="o">.</span><span class="n">index</span> <span class="o">=</span> <span class="n">s</span> |
| <span class="k">return</span> <span class="n">s</span> |
| |
| <span class="n">reindexed_expr</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'data_to_index'</span><span class="p">,</span> |
| <span class="n">data_to_index</span><span class="p">,</span> |
| <span class="p">[</span><span class="n">key</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(),</span> |
| <span class="p">)</span> |
| <span class="n">args</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">reindexed_expr</span><span class="p">]</span> |
| <span class="k">elif</span> <span class="n">callable</span><span class="p">(</span><span class="n">key</span><span class="p">):</span> |
| |
| <span class="k">def</span> <span class="nf">checked_callable_key</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="n">computed_index</span> <span class="o">=</span> <span class="n">key</span><span class="p">(</span><span class="n">df</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">computed_index</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span> |
| <span class="n">row_index</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">computed_index</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">row_index</span> <span class="o">=</span> <span class="n">computed_index</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">row_index</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> <span class="n">row_index</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span> |
| <span class="n">row_index</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="nb">bool</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">row_index</span><span class="p">))</span> |
| <span class="k">elif</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">row_index</span><span class="p">,</span> <span class="p">(</span><span class="nb">slice</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">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">row_index</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">computed_index</span> |
| |
| <span class="n">args</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="p">]</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">checked_callable_key</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="nb">type</span><span class="p">(</span><span class="n">key</span><span class="p">))</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'loc'</span><span class="p">,</span> |
| <span class="n">func</span><span class="p">,</span> |
| <span class="n">args</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="p">(</span> |
| <span class="n">partitionings</span><span class="o">.</span><span class="n">JoinIndex</span><span class="p">()</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">args</span><span class="p">)</span> <span class="o">></span> <span class="mi">1</span> |
| <span class="k">else</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="fm">__setitem__</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">not_implemented_method</span><span class="p">(</span> |
| <span class="s1">'loc.setitem'</span><span class="p">,</span> <span class="n">base_type</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexing</span><span class="o">.</span><span class="n">_LocIndexer</span><span class="p">)</span> |
| |
| <span class="nd">@populate_not_implemented</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexing</span><span class="o">.</span><span class="n">_iLocIndexer</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">_DeferredILoc</span><span class="p">(</span><span class="nb">object</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">frame</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_frame</span> <span class="o">=</span> <span class="n">frame</span> |
| |
| <span class="k">def</span> <span class="fm">__getitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">index</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span> |
| <span class="n">rows</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">index</span> |
| <span class="k">if</span> <span class="n">rows</span> <span class="o">!=</span> <span class="nb">slice</span><span class="p">(</span><span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"Using iloc to select rows is not supported because it's "</span> |
| <span class="s2">"position-based indexing is sensitive to the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'iloc'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">iloc</span><span class="p">[</span><span class="n">index</span><span class="p">],</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_frame</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"Using iloc to select rows is not supported because it's "</span> |
| <span class="s2">"position-based indexing is sensitive to the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="fm">__setitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">index</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"Using iloc to mutate a frame is not supported because it's "</span> |
| <span class="s2">"position-based indexing is sensitive to the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_DeferredStringMethods</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">cat</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">others</span><span class="p">,</span> <span class="n">join</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""If defined, ``others`` must be a :class:`DeferredSeries` or a ``list`` of</span> |
| <span class="sd"> ``DeferredSeries``."""</span> |
| <span class="k">if</span> <span class="n">others</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># Concatenate series into a single String</span> |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">(</span><span class="n">reason</span><span class="o">=</span><span class="p">(</span> |
| <span class="s2">"cat(others=None) concatenates all data in a Series into a single "</span> |
| <span class="s2">"string, so it requires collecting all data on a single node."</span> |
| <span class="p">))</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="n">df</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">cat</span><span class="p">(</span><span class="n">join</span><span class="o">=</span><span class="n">join</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="n">args</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">]</span> |
| |
| <span class="k">elif</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">others</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">)</span> <span class="ow">or</span> |
| <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">others</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> |
| <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">)</span> <span class="k">for</span> <span class="n">other</span> <span class="ow">in</span> <span class="n">others</span><span class="p">))):</span> |
| |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">others</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="n">others</span> <span class="o">=</span> <span class="p">[</span><span class="n">others</span><span class="p">]</span> |
| |
| <span class="n">requires</span> <span class="o">=</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">()</span> |
| <span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">args</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">cat</span><span class="p">(</span><span class="n">others</span><span class="o">=</span><span class="n">args</span><span class="p">[</span><span class="mi">1</span><span class="p">:],</span> <span class="n">join</span><span class="o">=</span><span class="n">join</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="n">args</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">]</span> <span class="o">+</span> <span class="p">[</span><span class="n">other</span><span class="o">.</span><span class="n">_expr</span> <span class="k">for</span> <span class="n">other</span> <span class="ow">in</span> <span class="n">others</span><span class="p">]</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"others must be None, DeferredSeries, or List[DeferredSeries] "</span> |
| <span class="sa">f</span><span class="s2">"(encountered </span><span class="si">{</span><span class="nb">type</span><span class="p">(</span><span class="n">others</span><span class="p">)</span><span class="si">}</span><span class="s2">). Other types are not supported "</span> |
| <span class="s2">"because they make this operation sensitive to the order of the "</span> |
| <span class="s2">"data."</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'cat'</span><span class="p">,</span> |
| <span class="n">func</span><span class="p">,</span> |
| <span class="n">args</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">requires</span><span class="p">,</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">repeat</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">repeats</span><span class="p">):</span> |
| <span class="sd">"""``repeats`` must be an ``int`` or a :class:`DeferredSeries`. Lists are</span> |
| <span class="sd"> not supported because they make this operation order-sensitive."""</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">repeats</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'repeat'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">series</span><span class="p">:</span> <span class="n">series</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">repeat</span><span class="p">(</span><span class="n">repeats</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="c1"># TODO(BEAM-11155): Defer to pandas to compute this proxy.</span> |
| <span class="c1"># Currently it incorrectly infers dtype bool, may require upstream</span> |
| <span class="c1"># fix.</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">repeats</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'repeat'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">series</span><span class="p">,</span> <span class="n">repeats_series</span><span class="p">:</span> <span class="n">series</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">repeat</span><span class="p">(</span><span class="n">repeats_series</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">repeats</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="c1"># TODO(BEAM-11155): Defer to pandas to compute this proxy.</span> |
| <span class="c1"># Currently it incorrectly infers dtype bool, may require upstream</span> |
| <span class="c1"># fix.</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">(),</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">repeats</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"str.repeat(repeats=) repeats must be an int or a DeferredSeries. "</span> |
| <span class="s2">"Lists are not supported because they make this operation sensitive "</span> |
| <span class="s2">"to the order of the data."</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</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">"str.repeat(repeats=) value must be an int or a "</span> |
| <span class="sa">f</span><span class="s2">"DeferredSeries (encountered </span><span class="si">{</span><span class="nb">type</span><span class="p">(</span><span class="n">repeats</span><span class="p">)</span><span class="si">}</span><span class="s2">)."</span><span class="p">)</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">get_dummies</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""</span> |
| <span class="sd"> Series must be categorical dtype. Please cast to ``CategoricalDtype``</span> |
| <span class="sd"> to ensure correct categories.</span> |
| <span class="sd"> """</span> |
| <span class="n">dtype</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dtype</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">dtype</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">CategoricalDtype</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"get_dummies() of non-categorical type is not supported because "</span> |
| <span class="s2">"the type of the output column depends on the data. Please use "</span> |
| <span class="s2">"pd.CategoricalDtype with explicit categories."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| |
| <span class="n">split_cats</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">cat</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="n">sep</span><span class="o">=</span><span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'sep'</span><span class="p">,</span> <span class="s1">'|'</span><span class="p">))</span> <span class="k">for</span> <span class="n">cat</span> <span class="ow">in</span> <span class="n">dtype</span><span class="o">.</span><span class="n">categories</span> |
| <span class="p">]</span> |
| |
| <span class="c1"># Adding the nan category because the there could be the case that</span> |
| <span class="c1"># the data includes NaNs, which is not valid to be casted as a Category,</span> |
| <span class="c1"># but nevertheless would be broadcasted as a column in get_dummies()</span> |
| <span class="n">columns</span> <span class="o">=</span> <span class="nb">sorted</span><span class="p">(</span><span class="nb">set</span><span class="p">()</span><span class="o">.</span><span class="n">union</span><span class="p">(</span><span class="o">*</span><span class="n">split_cats</span><span class="p">))</span> |
| <span class="n">columns</span> <span class="o">=</span> <span class="n">columns</span> <span class="o">+</span> <span class="p">[</span><span class="s1">'nan'</span><span class="p">]</span> <span class="k">if</span> <span class="s1">'nan'</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">columns</span> <span class="k">else</span> <span class="n">columns</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="n">columns</span><span class="p">)</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span><span class="nb">int</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'get_dummies'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">series</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">concat</span><span class="p">(</span> |
| <span class="p">[</span><span class="n">proxy</span><span class="p">,</span> <span class="n">series</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">get_dummies</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)]</span> |
| <span class="p">)</span><span class="o">.</span><span class="n">fillna</span><span class="p">(</span><span class="n">value</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">method</span><span class="o">=</span><span class="kc">None</span><span class="p">)</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span><span class="s1">'int64'</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="k">def</span> <span class="nf">_split_helper</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">rsplit</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="n">expand</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s1">'expand'</span><span class="p">,</span> <span class="kc">False</span><span class="p">)</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">expand</span><span class="p">:</span> |
| <span class="c1"># Not creating separate columns</span> |
| <span class="n">proxy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">rsplit</span><span class="p">:</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">concat</span><span class="p">([</span><span class="n">proxy</span><span class="p">,</span> <span class="n">s</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)])</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">func</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">concat</span><span class="p">([</span><span class="n">proxy</span><span class="p">,</span> <span class="n">s</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">rsplit</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)])</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Creating separate columns, so must be more strict on dtype</span> |
| <span class="n">dtype</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dtype</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">dtype</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">CategoricalDtype</span><span class="p">):</span> |
| <span class="n">method_name</span> <span class="o">=</span> <span class="s1">'rsplit'</span> <span class="k">if</span> <span class="n">rsplit</span> <span class="k">else</span> <span class="s1">'split'</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"</span><span class="si">{</span><span class="n">method_name</span><span class="si">}</span><span class="s2">() of non-categorical type is not supported because "</span> |
| <span class="s2">"the type of the output column depends on the data. Please use "</span> |
| <span class="s2">"pd.CategoricalDtype with explicit categories."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| |
| <span class="c1"># Split the categories</span> |
| <span class="n">split_cats</span> <span class="o">=</span> <span class="n">dtype</span><span class="o">.</span><span class="n">categories</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="c1"># Count the number of new columns to create for proxy</span> |
| <span class="n">max_splits</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="nb">max</span><span class="p">(</span><span class="n">split_cats</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="nb">len</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="nb">range</span><span class="p">(</span><span class="n">max_splits</span><span class="p">))</span> |
| |
| <span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">rsplit</span><span class="p">:</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">s</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="n">s</span><span class="o">.</span><span class="n">str</span><span class="o">.</span><span class="n">rsplit</span><span class="p">(</span><span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| <span class="n">result</span><span class="p">[</span><span class="o">~</span><span class="n">result</span><span class="o">.</span><span class="n">isna</span><span class="p">()]</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">nan</span><span class="p">,</span> <span class="n">value</span><span class="o">=</span><span class="kc">None</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">result</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'split'</span><span class="p">,</span> |
| <span class="n">func</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">proxy</span><span class="o">=</span><span class="n">proxy</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">split</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""</span> |
| <span class="sd"> Like other non-deferred methods, dtype must be CategoricalDtype.</span> |
| <span class="sd"> One exception is when ``expand`` is ``False``. Because we are not</span> |
| <span class="sd"> creating new columns at construction time, dtype can be `str`.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_split_helper</span><span class="p">(</span><span class="n">rsplit</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="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">args_to_kwargs</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">populate_defaults</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">rsplit</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""</span> |
| <span class="sd"> Like other non-deferred methods, dtype must be CategoricalDtype.</span> |
| <span class="sd"> One exception is when ``expand`` is ``False``. Because we are not</span> |
| <span class="sd"> creating new columns at construction time, dtype can be `str`.</span> |
| <span class="sd"> """</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_split_helper</span><span class="p">(</span><span class="n">rsplit</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| |
| <span class="n">ELEMENTWISE_STRING_METHODS</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'capitalize'</span><span class="p">,</span> |
| <span class="s1">'casefold'</span><span class="p">,</span> |
| <span class="s1">'contains'</span><span class="p">,</span> |
| <span class="s1">'count'</span><span class="p">,</span> |
| <span class="s1">'endswith'</span><span class="p">,</span> |
| <span class="s1">'extract'</span><span class="p">,</span> |
| <span class="s1">'findall'</span><span class="p">,</span> |
| <span class="s1">'fullmatch'</span><span class="p">,</span> |
| <span class="s1">'get'</span><span class="p">,</span> |
| <span class="s1">'isalnum'</span><span class="p">,</span> |
| <span class="s1">'isalpha'</span><span class="p">,</span> |
| <span class="s1">'isdecimal'</span><span class="p">,</span> |
| <span class="s1">'isdigit'</span><span class="p">,</span> |
| <span class="s1">'islower'</span><span class="p">,</span> |
| <span class="s1">'isnumeric'</span><span class="p">,</span> |
| <span class="s1">'isspace'</span><span class="p">,</span> |
| <span class="s1">'istitle'</span><span class="p">,</span> |
| <span class="s1">'isupper'</span><span class="p">,</span> |
| <span class="s1">'join'</span><span class="p">,</span> |
| <span class="s1">'len'</span><span class="p">,</span> |
| <span class="s1">'lower'</span><span class="p">,</span> |
| <span class="s1">'lstrip'</span><span class="p">,</span> |
| <span class="s1">'match'</span><span class="p">,</span> |
| <span class="s1">'pad'</span><span class="p">,</span> |
| <span class="s1">'partition'</span><span class="p">,</span> |
| <span class="s1">'removeprefix'</span><span class="p">,</span> |
| <span class="s1">'removesuffix'</span><span class="p">,</span> |
| <span class="s1">'replace'</span><span class="p">,</span> |
| <span class="s1">'rpartition'</span><span class="p">,</span> |
| <span class="s1">'rstrip'</span><span class="p">,</span> |
| <span class="s1">'slice'</span><span class="p">,</span> |
| <span class="s1">'slice_replace'</span><span class="p">,</span> |
| <span class="s1">'startswith'</span><span class="p">,</span> |
| <span class="s1">'strip'</span><span class="p">,</span> |
| <span class="s1">'swapcase'</span><span class="p">,</span> |
| <span class="s1">'title'</span><span class="p">,</span> |
| <span class="s1">'upper'</span><span class="p">,</span> |
| <span class="s1">'wrap'</span><span class="p">,</span> |
| <span class="s1">'zfill'</span><span class="p">,</span> |
| <span class="s1">'__getitem__'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="n">NON_ELEMENTWISE_STRING_METHODS</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'extractall'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="k">def</span> <span class="nf">make_str_func</span><span class="p">(</span><span class="n">method</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">df_str</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">str</span> |
| <span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span> |
| <span class="c1"># If there's a non-string value in a Series passed to .str method, pandas</span> |
| <span class="c1"># will generally just replace it with NaN in the result. However if</span> |
| <span class="c1"># there are _only_ non-string values, pandas will raise:</span> |
| <span class="c1">#</span> |
| <span class="c1"># AttributeError: Can only use .str accessor with string values!</span> |
| <span class="c1">#</span> |
| <span class="c1"># This can happen to us at execution time if we split a partition that is</span> |
| <span class="c1"># only non-strings. This branch just replaces all those values with NaN</span> |
| <span class="c1"># in that case.</span> |
| <span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">_</span><span class="p">:</span> <span class="n">np</span><span class="o">.</span><span class="n">nan</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">df_str</span><span class="p">,</span> <span class="n">method</span><span class="p">)(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">func</span> |
| |
| <span class="k">for</span> <span class="n">method</span> <span class="ow">in</span> <span class="n">ELEMENTWISE_STRING_METHODS</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">,</span> <span class="n">method</span><span class="p">):</span> |
| <span class="c1"># older versions (1.0.x) don't support some of these methods</span> |
| <span class="k">continue</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">_DeferredStringMethods</span><span class="p">,</span> |
| <span class="n">method</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">make_str_func</span><span class="p">(</span><span class="n">method</span><span class="p">),</span> |
| <span class="n">name</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">))</span> |
| |
| <span class="k">for</span> <span class="n">method</span> <span class="ow">in</span> <span class="n">NON_ELEMENTWISE_STRING_METHODS</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">,</span> <span class="n">method</span><span class="p">):</span> |
| <span class="c1"># older versions (1.0.x) don't support some of these methods</span> |
| <span class="k">continue</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">_DeferredStringMethods</span><span class="p">,</span> |
| <span class="n">method</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_proxy_method</span><span class="p">(</span> |
| <span class="n">make_str_func</span><span class="p">(</span><span class="n">method</span><span class="p">),</span> |
| <span class="n">name</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">strings</span><span class="o">.</span><span class="n">StringMethods</span><span class="p">,</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Singleton</span><span class="p">()))</span> |
| |
| |
| <span class="k">def</span> <span class="nf">make_cat_func</span><span class="p">(</span><span class="n">method</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">df</span><span class="o">.</span><span class="n">cat</span><span class="p">,</span> <span class="n">method</span><span class="p">)(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">func</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_DeferredCategoricalMethods</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">arrays</span><span class="o">.</span><span class="n">categorical</span><span class="o">.</span><span class="n">CategoricalAccessor</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">categories</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">cat</span><span class="o">.</span><span class="n">categories</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">arrays</span><span class="o">.</span><span class="n">categorical</span><span class="o">.</span><span class="n">CategoricalAccessor</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">ordered</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">cat</span><span class="o">.</span><span class="n">ordered</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">arrays</span><span class="o">.</span><span class="n">categorical</span><span class="o">.</span><span class="n">CategoricalAccessor</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">codes</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'codes'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">cat</span><span class="o">.</span><span class="n">codes</span><span class="p">,</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="p">)</span> |
| <span class="p">)</span> |
| |
| <span class="n">remove_unused_categories</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">arrays</span><span class="o">.</span><span class="n">categorical</span><span class="o">.</span><span class="n">CategoricalAccessor</span><span class="p">,</span> |
| <span class="s1">'remove_unused_categories'</span><span class="p">,</span> <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-columns"</span><span class="p">)</span> |
| |
| <span class="n">ELEMENTWISE_CATEGORICAL_METHODS</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'add_categories'</span><span class="p">,</span> |
| <span class="s1">'as_ordered'</span><span class="p">,</span> |
| <span class="s1">'as_unordered'</span><span class="p">,</span> |
| <span class="s1">'remove_categories'</span><span class="p">,</span> |
| <span class="s1">'rename_categories'</span><span class="p">,</span> |
| <span class="s1">'reorder_categories'</span><span class="p">,</span> |
| <span class="s1">'set_categories'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="k">for</span> <span class="n">method</span> <span class="ow">in</span> <span class="n">ELEMENTWISE_CATEGORICAL_METHODS</span><span class="p">:</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">_DeferredCategoricalMethods</span><span class="p">,</span> |
| <span class="n">method</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span> |
| <span class="n">make_cat_func</span><span class="p">(</span><span class="n">method</span><span class="p">),</span> <span class="n">name</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">arrays</span><span class="o">.</span><span class="n">categorical</span><span class="o">.</span><span class="n">CategoricalAccessor</span><span class="p">))</span> |
| |
| <span class="k">class</span> <span class="nc">_DeferredDatetimeMethods</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredBase</span><span class="p">):</span> |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">tz</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dt</span><span class="o">.</span><span class="n">tz</span> |
| |
| <span class="nd">@property</span> <span class="c1"># type: ignore</span> |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">freq</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">()</span><span class="o">.</span><span class="n">dt</span><span class="o">.</span><span class="n">freq</span> |
| |
| <span class="nd">@frame_base</span><span class="o">.</span><span class="n">with_docs_from</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">)</span> |
| <span class="k">def</span> <span class="nf">tz_localize</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">ambiguous</span><span class="o">=</span><span class="s1">'infer'</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="sd">"""``ambiguous`` cannot be set to ``"infer"`` as its semantics are</span> |
| <span class="sd"> order-sensitive. Similarly, specifying ``ambiguous`` as an</span> |
| <span class="sd"> :class:`~numpy.ndarray` is order-sensitive, but you can achieve similar</span> |
| <span class="sd"> functionality by specifying ``ambiguous`` as a Series."""</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ambiguous</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="s2">"tz_localize(ambiguous=ndarray) is not supported because it makes "</span> |
| <span class="s2">"this operation sensitive to the order of the data. Please use a "</span> |
| <span class="s2">"DeferredSeries instead."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ambiguous</span><span class="p">,</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'tz_localize'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">,</span> |
| <span class="n">ambiguous</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">dt</span><span class="o">.</span><span class="n">tz_localize</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="n">ambiguous</span><span class="o">=</span><span class="n">ambiguous</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">,</span> <span class="n">ambiguous</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| <span class="k">elif</span> <span class="n">ambiguous</span> <span class="o">==</span> <span class="s1">'infer'</span><span class="p">:</span> |
| <span class="c1"># infer attempts to infer based on the order of the timestamps</span> |
| <span class="k">raise</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">WontImplementError</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"tz_localize(ambiguous=</span><span class="si">{</span><span class="n">ambiguous</span><span class="si">!r}</span><span class="s2">) is not allowed because it "</span> |
| <span class="s2">"makes this operation sensitive to the order of the data."</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"order-sensitive"</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">DeferredFrame</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">ComputedExpression</span><span class="p">(</span> |
| <span class="s1">'tz_localize'</span><span class="p">,</span> |
| <span class="k">lambda</span> <span class="n">s</span><span class="p">:</span> <span class="n">s</span><span class="o">.</span><span class="n">dt</span><span class="o">.</span><span class="n">tz_localize</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="n">ambiguous</span><span class="o">=</span><span class="n">ambiguous</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">),</span> |
| <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_expr</span><span class="p">],</span> |
| <span class="n">requires_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">(),</span> |
| <span class="n">preserves_partition_by</span><span class="o">=</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Arbitrary</span><span class="p">()))</span> |
| |
| |
| <span class="n">to_period</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">,</span> <span class="s1">'to_period'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"event-time-semantics"</span><span class="p">)</span> |
| <span class="n">to_pydatetime</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">,</span> <span class="s1">'to_pydatetime'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| <span class="n">to_pytimedelta</span> <span class="o">=</span> <span class="n">frame_base</span><span class="o">.</span><span class="n">wont_implement_method</span><span class="p">(</span> |
| <span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">,</span> <span class="s1">'to_pytimedelta'</span><span class="p">,</span> |
| <span class="n">reason</span><span class="o">=</span><span class="s2">"non-deferred-result"</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">make_dt_property</span><span class="p">(</span><span class="n">method</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">df</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">df</span><span class="o">.</span><span class="n">dt</span><span class="p">,</span> <span class="n">method</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">func</span> |
| |
| <span class="k">def</span> <span class="nf">make_dt_func</span><span class="p">(</span><span class="n">method</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">df</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">df</span><span class="o">.</span><span class="n">dt</span><span class="p">,</span> <span class="n">method</span><span class="p">)(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">func</span> |
| |
| |
| <span class="n">ELEMENTWISE_DATETIME_METHODS</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'ceil'</span><span class="p">,</span> |
| <span class="s1">'day_name'</span><span class="p">,</span> |
| <span class="s1">'month_name'</span><span class="p">,</span> |
| <span class="s1">'floor'</span><span class="p">,</span> |
| <span class="s1">'isocalendar'</span><span class="p">,</span> |
| <span class="s1">'round'</span><span class="p">,</span> |
| <span class="s1">'normalize'</span><span class="p">,</span> |
| <span class="s1">'strftime'</span><span class="p">,</span> |
| <span class="s1">'tz_convert'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="k">for</span> <span class="n">method</span> <span class="ow">in</span> <span class="n">ELEMENTWISE_DATETIME_METHODS</span><span class="p">:</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">,</span> <span class="n">method</span><span class="p">):</span> |
| <span class="c1"># older versions (1.0.x) don't support some of these methods</span> |
| <span class="k">continue</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">_DeferredDatetimeMethods</span><span class="p">,</span> |
| <span class="n">method</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span> |
| <span class="n">make_dt_func</span><span class="p">(</span><span class="n">method</span><span class="p">),</span> |
| <span class="n">name</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">))</span> |
| |
| <span class="n">ELEMENTWISE_DATETIME_PROPERTIES</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'date'</span><span class="p">,</span> |
| <span class="s1">'day'</span><span class="p">,</span> |
| <span class="s1">'dayofweek'</span><span class="p">,</span> |
| <span class="s1">'dayofyear'</span><span class="p">,</span> |
| <span class="s1">'days_in_month'</span><span class="p">,</span> |
| <span class="s1">'daysinmonth'</span><span class="p">,</span> |
| <span class="s1">'hour'</span><span class="p">,</span> |
| <span class="s1">'is_leap_year'</span><span class="p">,</span> |
| <span class="s1">'is_month_end'</span><span class="p">,</span> |
| <span class="s1">'is_month_start'</span><span class="p">,</span> |
| <span class="s1">'is_quarter_end'</span><span class="p">,</span> |
| <span class="s1">'is_quarter_start'</span><span class="p">,</span> |
| <span class="s1">'is_year_end'</span><span class="p">,</span> |
| <span class="s1">'is_year_start'</span><span class="p">,</span> |
| <span class="s1">'microsecond'</span><span class="p">,</span> |
| <span class="s1">'minute'</span><span class="p">,</span> |
| <span class="s1">'month'</span><span class="p">,</span> |
| <span class="s1">'nanosecond'</span><span class="p">,</span> |
| <span class="s1">'quarter'</span><span class="p">,</span> |
| <span class="s1">'second'</span><span class="p">,</span> |
| <span class="s1">'time'</span><span class="p">,</span> |
| <span class="s1">'timetz'</span><span class="p">,</span> |
| <span class="s1">'week'</span><span class="p">,</span> |
| <span class="s1">'weekday'</span><span class="p">,</span> |
| <span class="s1">'weekofyear'</span><span class="p">,</span> |
| <span class="s1">'year'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="k">for</span> <span class="n">method</span> <span class="ow">in</span> <span class="n">ELEMENTWISE_DATETIME_PROPERTIES</span><span class="p">:</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">_DeferredDatetimeMethods</span><span class="p">,</span> |
| <span class="n">method</span><span class="p">,</span> |
| <span class="nb">property</span><span class="p">(</span><span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span> |
| <span class="n">make_dt_property</span><span class="p">(</span><span class="n">method</span><span class="p">),</span> |
| <span class="n">name</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">core</span><span class="o">.</span><span class="n">indexes</span><span class="o">.</span><span class="n">accessors</span><span class="o">.</span><span class="n">DatetimeProperties</span><span class="p">)))</span> |
| |
| |
| <span class="k">for</span> <span class="n">base</span> <span class="ow">in</span> <span class="p">[</span><span class="s1">'add'</span><span class="p">,</span> |
| <span class="s1">'sub'</span><span class="p">,</span> |
| <span class="s1">'mul'</span><span class="p">,</span> |
| <span class="s1">'div'</span><span class="p">,</span> |
| <span class="s1">'truediv'</span><span class="p">,</span> |
| <span class="s1">'floordiv'</span><span class="p">,</span> |
| <span class="s1">'mod'</span><span class="p">,</span> |
| <span class="s1">'divmod'</span><span class="p">,</span> |
| <span class="s1">'pow'</span><span class="p">,</span> |
| <span class="s1">'and'</span><span class="p">,</span> |
| <span class="s1">'or'</span><span class="p">]:</span> |
| <span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="p">[</span><span class="s1">'</span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> <span class="s1">'r</span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> <span class="s1">'__</span><span class="si">%s</span><span class="s1">__'</span><span class="p">,</span> <span class="s1">'__r</span><span class="si">%s</span><span class="s1">__'</span><span class="p">]:</span> |
| <span class="c1"># TODO: non-trivial level?</span> |
| <span class="n">name</span> <span class="o">=</span> <span class="n">p</span> <span class="o">%</span> <span class="n">base</span> |
| <span class="k">if</span> <span class="nb">hasattr</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">name</span><span class="p">):</span> |
| <span class="nb">setattr</span><span class="p">(</span> |
| <span class="n">DeferredSeries</span><span class="p">,</span> |
| <span class="n">name</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">restrictions</span><span class="o">=</span><span class="p">{</span><span class="s1">'level'</span><span class="p">:</span> <span class="kc">None</span><span class="p">},</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">))</span> |
| <span class="k">if</span> <span class="nb">hasattr</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">name</span><span class="p">):</span> |
| <span class="nb">setattr</span><span class="p">(</span> |
| <span class="n">DeferredDataFrame</span><span class="p">,</span> |
| <span class="n">name</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">restrictions</span><span class="o">=</span><span class="p">{</span><span class="s1">'level'</span><span class="p">:</span> <span class="kc">None</span><span class="p">},</span> |
| <span class="n">base</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">inplace_name</span> <span class="o">=</span> <span class="s1">'__i</span><span class="si">%s</span><span class="s1">__'</span> <span class="o">%</span> <span class="n">base</span> |
| <span class="k">if</span> <span class="nb">hasattr</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">inplace_name</span><span class="p">):</span> |
| <span class="nb">setattr</span><span class="p">(</span> |
| <span class="n">DeferredSeries</span><span class="p">,</span> |
| <span class="n">inplace_name</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">inplace_name</span><span class="p">,</span> <span class="n">inplace</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">))</span> |
| <span class="k">if</span> <span class="nb">hasattr</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">inplace_name</span><span class="p">):</span> |
| <span class="nb">setattr</span><span class="p">(</span> |
| <span class="n">DeferredDataFrame</span><span class="p">,</span> |
| <span class="n">inplace_name</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">inplace_name</span><span class="p">,</span> <span class="n">inplace</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">base</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="k">for</span> <span class="n">name</span> <span class="ow">in</span> <span class="p">[</span><span class="s1">'lt'</span><span class="p">,</span> <span class="s1">'le'</span><span class="p">,</span> <span class="s1">'gt'</span><span class="p">,</span> <span class="s1">'ge'</span><span class="p">,</span> <span class="s1">'eq'</span><span class="p">,</span> <span class="s1">'ne'</span><span class="p">]:</span> |
| <span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="s1">'</span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> <span class="s1">'__</span><span class="si">%s</span><span class="s1">__'</span><span class="p">:</span> |
| <span class="c1"># Note that non-underscore name is used for both as the __xxx__ methods are</span> |
| <span class="c1"># order-sensitive.</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredSeries</span><span class="p">,</span> <span class="n">p</span> <span class="o">%</span> <span class="n">name</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">))</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredDataFrame</span><span class="p">,</span> <span class="n">p</span> <span class="o">%</span> <span class="n">name</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">base</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="k">for</span> <span class="n">name</span> <span class="ow">in</span> <span class="p">[</span><span class="s1">'__neg__'</span><span class="p">,</span> <span class="s1">'__pos__'</span><span class="p">,</span> <span class="s1">'__invert__'</span><span class="p">]:</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredSeries</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">base</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">))</span> |
| <span class="nb">setattr</span><span class="p">(</span><span class="n">DeferredDataFrame</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> |
| <span class="n">frame_base</span><span class="o">.</span><span class="n">_elementwise_method</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">base</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">DeferredSeries</span><span class="o">.</span><span class="n">multiply</span> <span class="o">=</span> <span class="n">DeferredSeries</span><span class="o">.</span><span class="n">mul</span> <span class="c1"># type: ignore</span> |
| <span class="n">DeferredDataFrame</span><span class="o">.</span><span class="n">multiply</span> <span class="o">=</span> <span class="n">DeferredDataFrame</span><span class="o">.</span><span class="n">mul</span> <span class="c1"># type: ignore</span> |
| <span class="n">DeferredSeries</span><span class="o">.</span><span class="n">subtract</span> <span class="o">=</span> <span class="n">DeferredSeries</span><span class="o">.</span><span class="n">sub</span> <span class="c1"># type: ignore</span> |
| <span class="n">DeferredDataFrame</span><span class="o">.</span><span class="n">subtract</span> <span class="o">=</span> <span class="n">DeferredDataFrame</span><span class="o">.</span><span class="n">sub</span> <span class="c1"># type: ignore</span> |
| <span class="n">DeferredSeries</span><span class="o">.</span><span class="n">divide</span> <span class="o">=</span> <span class="n">DeferredSeries</span><span class="o">.</span><span class="n">div</span> <span class="c1"># type: ignore</span> |
| <span class="n">DeferredDataFrame</span><span class="o">.</span><span class="n">divide</span> <span class="o">=</span> <span class="n">DeferredDataFrame</span><span class="o">.</span><span class="n">div</span> <span class="c1"># type: ignore</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_slice_parts</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="k">yield</span> <span class="n">s</span><span class="o">.</span><span class="n">start</span> |
| <span class="k">yield</span> <span class="n">s</span><span class="o">.</span><span class="n">stop</span> |
| <span class="k">yield</span> <span class="n">s</span><span class="o">.</span><span class="n">step</span> |
| |
| <span class="k">def</span> <span class="nf">_is_null_slice</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="nb">slice</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">all</span><span class="p">(</span><span class="n">x</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">_slice_parts</span><span class="p">(</span><span class="n">s</span><span class="p">))</span> |
| |
| <span class="k">def</span> <span class="nf">_is_integer_slice</span><span class="p">(</span><span class="n">s</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="nb">slice</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">all</span><span class="p">(</span> |
| <span class="n">x</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">_slice_parts</span><span class="p">(</span><span class="n">s</span><span class="p">))</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">_is_null_slice</span><span class="p">(</span><span class="n">s</span><span class="p">)</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> |