| |
| |
| <!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.transforms — Apache Beam 2.47.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.47.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.testing.html">apache_beam.testing package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.transforms.html">apache_beam.transforms package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.typehints.html">apache_beam.typehints package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.utils.html">apache_beam.utils package</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.yaml.html">apache_beam.yaml package</a></li> |
| </ul> |
| <ul> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.error.html">apache_beam.error module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pipeline.html">apache_beam.pipeline module</a></li> |
| <li class="toctree-l1"><a class="reference internal" href="../../../apache_beam.pvalue.html">apache_beam.pvalue module</a></li> |
| </ul> |
| |
| |
| |
| </div> |
| </div> |
| </nav> |
| |
| <section data-toggle="wy-nav-shift" class="wy-nav-content-wrap"> |
| |
| |
| <nav class="wy-nav-top" aria-label="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.transforms</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.transforms</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="kn">import</span> <span class="nn">collections</span> |
| <span class="kn">import</span> <span class="nn">logging</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TYPE_CHECKING</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Dict</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">Mapping</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Tuple</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">TypeVar</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Union</span> |
| |
| <span class="kn">import</span> <span class="nn">pandas</span> <span class="k">as</span> <span class="nn">pd</span> |
| |
| <span class="kn">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</span> |
| <span class="kn">from</span> <span class="nn">apache_beam</span> <span class="kn">import</span> <span class="n">transforms</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">frames</span> <span class="c1"># pylint: disable=unused-import</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="kn">from</span> <span class="nn">apache_beam.utils</span> <span class="kn">import</span> <span class="n">windowed_value</span> |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s1">'DataframeTransform'</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| <span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span> |
| <span class="c1"># pylint: disable=ungrouped-imports</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.pvalue</span> <span class="kn">import</span> <span class="n">PCollection</span> |
| |
| <span class="n">T</span> <span class="o">=</span> <span class="n">TypeVar</span><span class="p">(</span><span class="s1">'T'</span><span class="p">)</span> |
| |
| <span class="n">TARGET_PARTITION_SIZE</span> <span class="o">=</span> <span class="mi">1</span> <span class="o"><<</span> <span class="mi">23</span> <span class="c1"># 8M</span> |
| <span class="n">MIN_PARTITION_SIZE</span> <span class="o">=</span> <span class="mi">1</span> <span class="o"><<</span> <span class="mi">19</span> <span class="c1"># 0.5M</span> |
| <span class="n">MAX_PARTITIONS</span> <span class="o">=</span> <span class="mi">1000</span> |
| <span class="n">DEFAULT_PARTITIONS</span> <span class="o">=</span> <span class="mi">100</span> |
| <span class="n">MIN_PARTITIONS</span> <span class="o">=</span> <span class="mi">10</span> |
| <span class="n">PER_COL_OVERHEAD</span> <span class="o">=</span> <span class="mi">1000</span> |
| |
| |
| <div class="viewcode-block" id="DataframeTransform"><a class="viewcode-back" href="../../../apache_beam.dataframe.transforms.html#apache_beam.dataframe.transforms.DataframeTransform">[docs]</a><span class="k">class</span> <span class="nc">DataframeTransform</span><span class="p">(</span><span class="n">transforms</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A PTransform for applying function that takes and returns dataframes</span> |
| <span class="sd"> to one or more PCollections.</span> |
| |
| <span class="sd"> :class:`DataframeTransform` will accept a PCollection with a `schema`_ and</span> |
| <span class="sd"> batch it into :class:`~pandas.DataFrame` instances if necessary::</span> |
| |
| <span class="sd"> (pcoll | beam.Select(key=..., foo=..., bar=...)</span> |
| <span class="sd"> | DataframeTransform(lambda df: df.group_by('key').sum()))</span> |
| |
| <span class="sd"> It is also possible to process a PCollection of :class:`~pandas.DataFrame`</span> |
| <span class="sd"> instances directly, in this case a "proxy" must be provided. For example, if</span> |
| <span class="sd"> ``pcoll`` is a PCollection of DataFrames, one could write::</span> |
| |
| <span class="sd"> pcoll | DataframeTransform(lambda df: df.group_by('key').sum(), proxy=...)</span> |
| |
| <span class="sd"> To pass multiple PCollections, pass a tuple of PCollections wich will be</span> |
| <span class="sd"> passed to the callable as positional arguments, or a dictionary of</span> |
| <span class="sd"> PCollections, in which case they will be passed as keyword arguments.</span> |
| |
| <span class="sd"> Args:</span> |
| <span class="sd"> yield_elements: (optional, default: "schemas") If set to ``"pandas"``,</span> |
| <span class="sd"> return PCollection(s) containing the raw Pandas objects</span> |
| <span class="sd"> (:class:`~pandas.DataFrame` or :class:`~pandas.Series` as appropriate).</span> |
| <span class="sd"> If set to ``"schemas"``, return an element-wise PCollection, where</span> |
| <span class="sd"> DataFrame and Series instances are expanded to one element per row.</span> |
| <span class="sd"> DataFrames are converted to `schema-aware`_ PCollections, where column</span> |
| <span class="sd"> values can be accessed by attribute.</span> |
| <span class="sd"> include_indexes: (optional, default: False) When</span> |
| <span class="sd"> ``yield_elements="schemas"``, if ``include_indexes=True``, attempt to</span> |
| <span class="sd"> include index columns in the output schema for expanded DataFrames.</span> |
| <span class="sd"> Raises an error if any of the index levels are unnamed (name=None), or if</span> |
| <span class="sd"> any of the names are not unique among all column and index names.</span> |
| <span class="sd"> proxy: (optional) An empty :class:`~pandas.DataFrame` or</span> |
| <span class="sd"> :class:`~pandas.Series` instance with the same ``dtype`` and ``name``</span> |
| <span class="sd"> as the elements of the input PCollection. Required when input</span> |
| <span class="sd"> PCollection :class:`~pandas.DataFrame` or :class:`~pandas.Series`</span> |
| <span class="sd"> elements. Ignored when input PCollection has a `schema`_.</span> |
| |
| <span class="sd"> .. _schema:</span> |
| <span class="sd"> https://beam.apache.org/documentation/programming-guide/#what-is-a-schema</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"> """</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">func</span><span class="p">,</span> <span class="n">proxy</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">yield_elements</span><span class="o">=</span><span class="s2">"schemas"</span><span class="p">,</span> <span class="n">include_indexes</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_func</span> <span class="o">=</span> <span class="n">func</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_proxy</span> <span class="o">=</span> <span class="n">proxy</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_yield_elements</span> <span class="o">=</span> <span class="n">yield_elements</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_include_indexes</span> <span class="o">=</span> <span class="n">include_indexes</span> |
| |
| <div class="viewcode-block" id="DataframeTransform.expand"><a class="viewcode-back" href="../../../apache_beam.dataframe.transforms.html#apache_beam.dataframe.transforms.DataframeTransform.expand">[docs]</a> <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">input_pcolls</span><span class="p">):</span> |
| <span class="c1"># Avoid circular import.</span> |
| <span class="kn">from</span> <span class="nn">apache_beam.dataframe</span> <span class="kn">import</span> <span class="n">convert</span> |
| |
| <span class="c1"># Convert inputs to a flat dict.</span> |
| <span class="n">input_dict</span> <span class="o">=</span> <span class="n">_flatten</span><span class="p">(</span><span class="n">input_pcolls</span><span class="p">)</span> <span class="c1"># type: Dict[Any, PCollection]</span> |
| <span class="n">proxies</span> <span class="o">=</span> <span class="n">_flatten</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_proxy</span><span class="p">)</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_proxy</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">else</span> <span class="p">{</span> |
| <span class="n">tag</span><span class="p">:</span> <span class="kc">None</span> |
| <span class="k">for</span> <span class="n">tag</span> <span class="ow">in</span> <span class="n">input_dict</span> |
| <span class="p">}</span> |
| <span class="n">input_frames</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">k</span><span class="p">:</span> <span class="n">convert</span><span class="o">.</span><span class="n">to_dataframe</span><span class="p">(</span><span class="n">pc</span><span class="p">,</span> <span class="n">proxies</span><span class="p">[</span><span class="n">k</span><span class="p">])</span> |
| <span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">pc</span> <span class="ow">in</span> <span class="n">input_dict</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="p">}</span> <span class="c1"># type: Dict[Any, DeferredFrame] # noqa: F821</span> |
| |
| <span class="c1"># Apply the function.</span> |
| <span class="n">frames_input</span> <span class="o">=</span> <span class="n">_substitute</span><span class="p">(</span><span class="n">input_pcolls</span><span class="p">,</span> <span class="n">input_frames</span><span class="p">)</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">frames_input</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span> |
| <span class="n">result_frames</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_func</span><span class="p">(</span><span class="o">**</span><span class="n">frames_input</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">frames_input</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span> |
| <span class="n">result_frames</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_func</span><span class="p">(</span><span class="o">*</span><span class="n">frames_input</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">result_frames</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_func</span><span class="p">(</span><span class="n">frames_input</span><span class="p">)</span> |
| |
| <span class="c1"># Compute results as a tuple.</span> |
| <span class="n">result_frames_dict</span> <span class="o">=</span> <span class="n">_flatten</span><span class="p">(</span><span class="n">result_frames</span><span class="p">)</span> |
| <span class="n">keys</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">result_frames_dict</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> |
| <span class="n">result_frames_tuple</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">result_frames_dict</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="k">for</span> <span class="n">key</span> <span class="ow">in</span> <span class="n">keys</span><span class="p">)</span> |
| <span class="n">result_pcolls_tuple</span> <span class="o">=</span> <span class="n">convert</span><span class="o">.</span><span class="n">to_pcollection</span><span class="p">(</span> |
| <span class="o">*</span><span class="n">result_frames_tuple</span><span class="p">,</span> |
| <span class="n">label</span><span class="o">=</span><span class="s1">'Eval'</span><span class="p">,</span> |
| <span class="n">always_return_tuple</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> |
| <span class="n">yield_elements</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_yield_elements</span><span class="p">,</span> |
| <span class="n">include_indexes</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_include_indexes</span><span class="p">)</span> |
| |
| <span class="c1"># Convert back to the structure returned by self._func.</span> |
| <span class="n">result_pcolls_dict</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">(</span><span class="nb">zip</span><span class="p">(</span><span class="n">keys</span><span class="p">,</span> <span class="n">result_pcolls_tuple</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">_substitute</span><span class="p">(</span><span class="n">result_frames</span><span class="p">,</span> <span class="n">result_pcolls_dict</span><span class="p">)</span></div></div> |
| |
| |
| <span class="k">class</span> <span class="nc">_DataframeExpressionsTransform</span><span class="p">(</span><span class="n">transforms</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">outputs</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_outputs</span> <span class="o">=</span> <span class="n">outputs</span> |
| |
| <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">inputs</span><span class="p">):</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_deferred_ops</span><span class="p">(</span><span class="n">inputs</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_outputs</span><span class="p">)</span> |
| |
| <span class="k">def</span> <span class="nf">_apply_deferred_ops</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">inputs</span><span class="p">,</span> <span class="c1"># type: Dict[expressions.Expression, PCollection]</span> |
| <span class="n">outputs</span><span class="p">,</span> <span class="c1"># type: Dict[Any, expressions.Expression]</span> |
| <span class="p">):</span> <span class="c1"># -> Dict[Any, PCollection]</span> |
| <span class="w"> </span><span class="sd">"""Construct a Beam graph that evaluates a set of expressions on a set of</span> |
| <span class="sd"> input PCollections.</span> |
| |
| <span class="sd"> :param inputs: A mapping of placeholder expressions to PCollections.</span> |
| <span class="sd"> :param outputs: A mapping of keys to expressions defined in terms of the</span> |
| <span class="sd"> placeholders of inputs.</span> |
| |
| <span class="sd"> Returns a dictionary whose keys are those of outputs, and whose values are</span> |
| <span class="sd"> PCollections corresponding to the values of outputs evaluated at the</span> |
| <span class="sd"> values of inputs.</span> |
| |
| <span class="sd"> Logically, `_apply_deferred_ops({x: a, y: b}, {f: F(x, y), g: G(x, y)})`</span> |
| <span class="sd"> returns `{f: F(a, b), g: G(a, b)}`.</span> |
| <span class="sd"> """</span> |
| <span class="k">class</span> <span class="nc">ComputeStage</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""A helper transform that computes a single stage of operations.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">stage</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">stage</span> <span class="o">=</span> <span class="n">stage</span> |
| |
| <span class="k">def</span> <span class="nf">default_label</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">return</span> <span class="s1">'</span><span class="si">%s</span><span class="s1">:</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="o">.</span><span class="n">ops</span><span class="p">,</span> <span class="nb">id</span><span class="p">(</span><span class="bp">self</span><span class="p">))</span> |
| |
| <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcolls</span><span class="p">):</span> |
| <span class="n">logging</span><span class="o">.</span><span class="n">info</span><span class="p">(</span><span class="s1">'Computing dataframe stage </span><span class="si">%s</span><span class="s1"> for </span><span class="si">%s</span><span class="s1">'</span><span class="p">,</span> <span class="bp">self</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="p">)</span> |
| <span class="n">scalar_inputs</span> <span class="o">=</span> <span class="p">[</span><span class="n">expr</span> <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="o">.</span><span class="n">inputs</span> <span class="k">if</span> <span class="n">is_scalar</span><span class="p">(</span><span class="n">expr</span><span class="p">)]</span> |
| <span class="n">tabular_inputs</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">expr</span> <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="o">.</span><span class="n">inputs</span> <span class="k">if</span> <span class="ow">not</span> <span class="n">is_scalar</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span> |
| <span class="p">]</span> |
| |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">tabular_inputs</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="n">partitioned_pcoll</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="nb">iter</span><span class="p">(</span> |
| <span class="n">pcolls</span><span class="o">.</span><span class="n">values</span><span class="p">()))</span><span class="o">.</span><span class="n">pipeline</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Create</span><span class="p">([{}])</span> |
| |
| <span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="o">.</span><span class="n">partitioning</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 required for these operations.</span> |
| <span class="c1"># Compute the number of partitions to use for the inputs based on</span> |
| <span class="c1"># the estimated size of the inputs.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="o">.</span><span class="n">partitioning</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="c1"># Always a single partition, don't waste time computing sizes.</span> |
| <span class="n">num_partitions</span> <span class="o">=</span> <span class="mi">1</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Estimate the sizes from the outputs of a *previous* stage such</span> |
| <span class="c1"># that using these estimates will not cause a fusion break.</span> |
| <span class="n">input_sizes</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="n">estimate_size</span><span class="p">(</span><span class="nb">input</span><span class="p">,</span> <span class="n">same_stage_ok</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span> |
| <span class="k">for</span> <span class="nb">input</span> <span class="ow">in</span> <span class="n">tabular_inputs</span> |
| <span class="p">]</span> |
| <span class="k">if</span> <span class="kc">None</span> <span class="ow">in</span> <span class="n">input_sizes</span><span class="p">:</span> |
| <span class="c1"># We were unable to (cheaply) compute the size of one or more</span> |
| <span class="c1"># inputs.</span> |
| <span class="n">num_partitions</span> <span class="o">=</span> <span class="n">DEFAULT_PARTITIONS</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">num_partitions</span> <span class="o">=</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">AsSingleton</span><span class="p">(</span> |
| <span class="n">input_sizes</span> |
| <span class="o">|</span> <span class="s1">'FlattenSizes'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Flatten</span><span class="p">()</span> |
| <span class="o">|</span> <span class="s1">'SumSizes'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">CombineGlobally</span><span class="p">(</span><span class="nb">sum</span><span class="p">)</span> |
| <span class="o">|</span> <span class="s1">'NumPartitions'</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span> |
| <span class="k">lambda</span> <span class="n">size</span><span class="p">:</span> <span class="nb">max</span><span class="p">(</span> |
| <span class="n">MIN_PARTITIONS</span><span class="p">,</span> |
| <span class="nb">min</span><span class="p">(</span><span class="n">MAX_PARTITIONS</span><span class="p">,</span> <span class="n">size</span> <span class="o">//</span> <span class="n">TARGET_PARTITION_SIZE</span><span class="p">))))</span> |
| |
| <span class="n">partition_fn</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="o">.</span><span class="n">partitioning</span><span class="o">.</span><span class="n">partition_fn</span> |
| |
| <span class="k">class</span> <span class="nc">Partition</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">PTransform</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">expand</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pcoll</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="n">pcoll</span> |
| <span class="c1"># Attempt to create batches of reasonable size.</span> |
| <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="n">_PreBatch</span><span class="p">())</span> |
| <span class="c1"># Actually partition.</span> |
| <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">FlatMap</span><span class="p">(</span><span class="n">partition_fn</span><span class="p">,</span> <span class="n">num_partitions</span><span class="p">)</span> |
| <span class="c1"># Don't bother shuffling empty partitions.</span> |
| <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Filter</span><span class="p">(</span><span class="k">lambda</span> <span class="n">k_df</span><span class="p">:</span> <span class="nb">len</span><span class="p">(</span><span class="n">k_df</span><span class="p">[</span><span class="mi">1</span><span class="p">])))</span> |
| |
| <span class="c1"># Arrange such that partitioned_pcoll is properly partitioned.</span> |
| <span class="n">main_pcolls</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">:</span> <span class="n">pcolls</span><span class="p">[</span><span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">]</span> <span class="o">|</span> <span class="s1">'Partition_</span><span class="si">%s</span><span class="s1">_</span><span class="si">%s</span><span class="s1">'</span> <span class="o">%</span> |
| <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="o">.</span><span class="n">partitioning</span><span class="p">,</span> <span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">)</span> <span class="o">>></span> <span class="n">Partition</span><span class="p">()</span> |
| <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">tabular_inputs</span> |
| <span class="p">}</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">CoGroupByKey</span><span class="p">()</span> |
| <span class="n">partitioned_pcoll</span> <span class="o">=</span> <span class="n">main_pcolls</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">ParDo</span><span class="p">(</span><span class="n">_ReBatch</span><span class="p">())</span> |
| |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Already partitioned, or no partitioning needed.</span> |
| <span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">tabular_inputs</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span> |
| <span class="n">tag</span> <span class="o">=</span> <span class="n">tabular_inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">_id</span> |
| <span class="n">partitioned_pcoll</span> <span class="o">=</span> <span class="n">pcolls</span><span class="p">[</span><span class="n">tag</span><span class="p">]</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">df</span><span class="p">:</span> <span class="p">{</span><span class="n">tag</span><span class="p">:</span> <span class="n">df</span><span class="p">})</span> |
| |
| <span class="n">side_pcolls</span> <span class="o">=</span> <span class="p">{</span> |
| <span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">:</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">AsSingleton</span><span class="p">(</span><span class="n">pcolls</span><span class="p">[</span><span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">])</span> |
| <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">scalar_inputs</span> |
| <span class="p">}</span> |
| |
| <span class="c1"># Actually evaluate the expressions.</span> |
| <span class="k">def</span> <span class="nf">evaluate</span><span class="p">(</span><span class="n">partition</span><span class="p">,</span> <span class="n">stage</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">stage</span><span class="p">,</span> <span class="o">**</span><span class="n">side_inputs</span><span class="p">):</span> |
| <span class="k">def</span> <span class="nf">lookup</span><span class="p">(</span><span class="n">expr</span><span class="p">):</span> |
| <span class="c1"># Use proxy if there's no data in this partition</span> |
| <span class="k">return</span> <span class="n">expr</span><span class="o">.</span><span class="n">proxy</span><span class="p">(</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">if</span> <span class="n">partition</span><span class="p">[</span><span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">]</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="n">partition</span><span class="p">[</span><span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">]</span> |
| |
| <span class="n">session</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">Session</span><span class="p">(</span> |
| <span class="nb">dict</span><span class="p">([(</span><span class="n">expr</span><span class="p">,</span> <span class="n">lookup</span><span class="p">(</span><span class="n">expr</span><span class="p">))</span> <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">tabular_inputs</span><span class="p">]</span> <span class="o">+</span> |
| <span class="p">[(</span><span class="n">expr</span><span class="p">,</span> <span class="n">side_inputs</span><span class="p">[</span><span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">])</span> <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">scalar_inputs</span><span class="p">]))</span> |
| <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">stage</span><span class="o">.</span><span class="n">outputs</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">beam</span><span class="o">.</span><span class="n">pvalue</span><span class="o">.</span><span class="n">TaggedOutput</span><span class="p">(</span><span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">,</span> <span class="n">expr</span><span class="o">.</span><span class="n">evaluate_at</span><span class="p">(</span><span class="n">session</span><span class="p">))</span> |
| |
| <span class="k">return</span> <span class="n">partitioned_pcoll</span> <span class="o">|</span> <span class="n">beam</span><span class="o">.</span><span class="n">FlatMap</span><span class="p">(</span><span class="n">evaluate</span><span class="p">,</span> <span class="o">**</span> |
| <span class="n">side_pcolls</span><span class="p">)</span><span class="o">.</span><span class="n">with_outputs</span><span class="p">()</span> |
| |
| <span class="k">class</span> <span class="nc">Stage</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Used to build up a set of operations that can be fused together.</span> |
| |
| <span class="sd"> Note that these Dataframe "stages" contain a CoGBK and hence are often</span> |
| <span class="sd"> split across multiple "executable" stages.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">inputs</span><span class="p">,</span> <span class="n">partitioning</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">inputs</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">inputs</span><span class="p">)</span> |
| <span class="k">if</span> <span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">inputs</span><span class="p">)</span> <span class="o">></span> <span class="mi">1</span> <span class="ow">and</span> |
| <span class="n">partitioning</span><span class="o">.</span><span class="n">is_subpartitioning_of</span><span class="p">(</span><span class="n">partitionings</span><span class="o">.</span><span class="n">Index</span><span class="p">())):</span> |
| <span class="c1"># We have to shuffle to co-locate, might as well partition.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">partitioning</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">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">partitioning</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="c1"># Not an actionable partitioning, use index.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">partitioning</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="bp">self</span><span class="o">.</span><span class="n">partitioning</span> <span class="o">=</span> <span class="n">partitioning</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">ops</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">outputs</span> <span class="o">=</span> <span class="nb">set</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="n">indent</span><span class="o">=</span><span class="mi">0</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">indent</span><span class="p">:</span> |
| <span class="n">sep</span> <span class="o">=</span> <span class="s1">'</span><span class="se">\n</span><span class="s1">'</span> <span class="o">+</span> <span class="s1">' '</span> <span class="o">*</span> <span class="n">indent</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">sep</span> <span class="o">=</span> <span class="s1">''</span> |
| <span class="k">return</span> <span class="p">(</span> |
| <span class="s2">"Stage[</span><span class="si">%s</span><span class="s2">inputs=</span><span class="si">%s</span><span class="s2">, </span><span class="si">%s</span><span class="s2">partitioning=</span><span class="si">%s</span><span class="s2">, </span><span class="si">%s</span><span class="s2">ops=</span><span class="si">%s</span><span class="s2">, </span><span class="si">%s</span><span class="s2">outputs=</span><span class="si">%s</span><span class="s2">]"</span> <span class="o">%</span> <span class="p">(</span> |
| <span class="n">sep</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">inputs</span><span class="p">,</span> |
| <span class="n">sep</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">partitioning</span><span class="p">,</span> |
| <span class="n">sep</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">ops</span><span class="p">,</span> |
| <span class="n">sep</span><span class="p">,</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">outputs</span><span class="p">))</span> |
| |
| <span class="c1"># First define some helper functions.</span> |
| <span class="k">def</span> <span class="nf">output_partitioning_in_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">stage</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Return the output partitioning of expr when computed in stage,</span> |
| <span class="sd"> or returns None if the expression cannot be computed in this stage.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="nf">maybe_upgrade_to_join_index</span><span class="p">(</span><span class="n">partitioning</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">partitioning</span><span class="o">.</span><span class="n">is_subpartitioning_of</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">return</span> <span class="n">partitionings</span><span class="o">.</span><span class="n">JoinIndex</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">partitioning</span> |
| |
| <span class="k">if</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">stage</span><span class="o">.</span><span class="n">inputs</span> <span class="ow">or</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">:</span> |
| <span class="c1"># Inputs are all partitioned by stage.partitioning.</span> |
| <span class="k">return</span> <span class="n">maybe_upgrade_to_join_index</span><span class="p">(</span><span class="n">stage</span><span class="o">.</span><span class="n">partitioning</span><span class="p">)</span> |
| |
| <span class="c1"># Anything that's not an input must have arguments</span> |
| <span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">())</span> |
| |
| <span class="n">arg_partitionings</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span> |
| <span class="n">output_partitioning_in_stage</span><span class="p">(</span><span class="n">arg</span><span class="p">,</span> <span class="n">stage</span><span class="p">)</span> <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">()</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">is_scalar</span><span class="p">(</span><span class="n">arg</span><span class="p">))</span> |
| |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">arg_partitionings</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span> |
| <span class="c1"># All inputs are scalars, output partitioning isn't dependent on the</span> |
| <span class="c1"># input.</span> |
| <span class="k">return</span> <span class="n">maybe_upgrade_to_join_index</span><span class="p">(</span><span class="n">expr</span><span class="o">.</span><span class="n">preserves_partition_by</span><span class="p">())</span> |
| |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">arg_partitionings</span><span class="p">)</span> <span class="o">></span> <span class="mi">1</span><span class="p">:</span> |
| <span class="c1"># Arguments must be identically partitioned, can't compute this</span> |
| <span class="c1"># expression here.</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| |
| <span class="n">arg_partitioning</span> <span class="o">=</span> <span class="n">arg_partitionings</span><span class="o">.</span><span class="n">pop</span><span class="p">()</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="n">expr</span><span class="o">.</span><span class="n">requires_partition_by</span><span class="p">()</span><span class="o">.</span><span class="n">is_subpartitioning_of</span><span class="p">(</span> |
| <span class="n">arg_partitioning</span><span class="p">):</span> |
| <span class="c1"># Arguments aren't partitioned sufficiently for this expression</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| |
| <span class="k">return</span> <span class="n">maybe_upgrade_to_join_index</span><span class="p">(</span> |
| <span class="n">expressions</span><span class="o">.</span><span class="n">output_partitioning</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">arg_partitioning</span><span class="p">))</span> |
| |
| <span class="k">def</span> <span class="nf">is_computable_in_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">stage</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">output_partitioning_in_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">stage</span><span class="p">)</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> |
| |
| <span class="k">def</span> <span class="nf">common_stages</span><span class="p">(</span><span class="n">stage_lists</span><span class="p">):</span> |
| <span class="c1"># Set intersection, with a preference for earlier items in the list.</span> |
| <span class="k">if</span> <span class="n">stage_lists</span><span class="p">:</span> |
| <span class="k">for</span> <span class="n">stage</span> <span class="ow">in</span> <span class="n">stage_lists</span><span class="p">[</span><span class="mi">0</span><span class="p">]:</span> |
| <span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="n">stage</span> <span class="ow">in</span> <span class="n">other</span> <span class="k">for</span> <span class="n">other</span> <span class="ow">in</span> <span class="n">stage_lists</span><span class="p">[</span><span class="mi">1</span><span class="p">:]):</span> |
| <span class="k">yield</span> <span class="n">stage</span> |
| |
| <span class="nd">@_memoize</span> |
| <span class="k">def</span> <span class="nf">is_scalar</span><span class="p">(</span><span class="n">expr</span><span class="p">):</span> |
| <span class="k">return</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">expr</span><span class="o">.</span><span class="n">proxy</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">@_memoize</span> |
| <span class="k">def</span> <span class="nf">expr_to_stages</span><span class="p">(</span><span class="n">expr</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">:</span> |
| <span class="c1"># Don't create a stage for each input, but it is still useful to record</span> |
| <span class="c1"># what which stages inputs are available from.</span> |
| <span class="k">return</span> <span class="p">[]</span> |
| |
| <span class="c1"># First attempt to compute this expression as part of an existing stage,</span> |
| <span class="c1"># if possible.</span> |
| <span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="n">arg</span> <span class="ow">in</span> <span class="n">inputs</span> <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">()):</span> |
| <span class="c1"># All input arguments; try to pick a stage that already has as many</span> |
| <span class="c1"># of the inputs, correctly partitioned, as possible.</span> |
| <span class="n">inputs_by_stage</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">int</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">():</span> |
| <span class="k">for</span> <span class="n">stage</span> <span class="ow">in</span> <span class="n">expr_to_stages</span><span class="p">(</span><span class="n">arg</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">is_computable_in_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">stage</span><span class="p">):</span> |
| <span class="n">inputs_by_stage</span><span class="p">[</span><span class="n">stage</span><span class="p">]</span> <span class="o">+=</span> <span class="mi">1</span> <span class="o">+</span> <span class="mi">100</span> <span class="o">*</span> <span class="p">(</span> |
| <span class="n">expr</span><span class="o">.</span><span class="n">requires_partition_by</span><span class="p">()</span> <span class="o">==</span> <span class="n">stage</span><span class="o">.</span><span class="n">partitioning</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">inputs_by_stage</span><span class="p">:</span> |
| <span class="c1"># Take the stage with the largest count.</span> |
| <span class="n">stage</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="n">inputs_by_stage</span><span class="o">.</span><span class="n">items</span><span class="p">(),</span> <span class="n">key</span><span class="o">=</span><span class="k">lambda</span> <span class="n">kv</span><span class="p">:</span> <span class="n">kv</span><span class="p">[</span><span class="mi">1</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="n">stage</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Try to pick a stage that has all the available non-input expressions.</span> |
| <span class="c1"># TODO(robertwb): Baring any that have all of them, we could try and</span> |
| <span class="c1"># pick one that has the most, but we need to ensure it is not a</span> |
| <span class="c1"># predecessor of any of the missing argument's stages.</span> |
| <span class="k">for</span> <span class="n">stage</span> <span class="ow">in</span> <span class="n">common_stages</span><span class="p">([</span><span class="n">expr_to_stages</span><span class="p">(</span><span class="n">arg</span><span class="p">)</span> <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">()</span> |
| <span class="k">if</span> <span class="n">arg</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">]):</span> |
| <span class="k">if</span> <span class="n">is_computable_in_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">stage</span><span class="p">):</span> |
| <span class="k">break</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">stage</span> <span class="o">=</span> <span class="kc">None</span> |
| |
| <span class="k">if</span> <span class="n">stage</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="c1"># No stage available, compute this expression as part of a new stage.</span> |
| <span class="n">stage</span> <span class="o">=</span> <span class="n">Stage</span><span class="p">(</span><span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">(),</span> <span class="n">expr</span><span class="o">.</span><span class="n">requires_partition_by</span><span class="p">())</span> |
| <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">():</span> |
| <span class="c1"># For each argument, declare that it is also available in</span> |
| <span class="c1"># this new stage.</span> |
| <span class="n">expr_to_stages</span><span class="p">(</span><span class="n">arg</span><span class="p">)</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">stage</span><span class="p">)</span> |
| <span class="c1"># It also must be declared as an output of the producing stage.</span> |
| <span class="n">expr_to_stage</span><span class="p">(</span><span class="n">arg</span><span class="p">)</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">arg</span><span class="p">)</span> |
| <span class="n">stage</span><span class="o">.</span><span class="n">ops</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span> |
| <span class="c1"># Ensure that any inputs for the overall transform are added</span> |
| <span class="c1"># in downstream stages.</span> |
| <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">:</span> |
| <span class="n">stage</span><span class="o">.</span><span class="n">inputs</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">arg</span><span class="p">)</span> |
| <span class="c1"># This is a list as given expression may be available in many stages.</span> |
| <span class="k">return</span> <span class="p">[</span><span class="n">stage</span><span class="p">]</span> |
| |
| <span class="k">def</span> <span class="nf">expr_to_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">):</span> |
| <span class="c1"># Any will do; the first requires the fewest intermediate stages.</span> |
| <span class="k">return</span> <span class="n">expr_to_stages</span><span class="p">(</span><span class="n">expr</span><span class="p">)[</span><span class="mi">0</span><span class="p">]</span> |
| |
| <span class="c1"># Ensure each output is computed.</span> |
| <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">outputs</span><span class="o">.</span><span class="n">values</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">expr</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">:</span> |
| <span class="n">expr_to_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span><span class="o">.</span><span class="n">outputs</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span> |
| |
| <span class="nd">@_memoize</span> |
| <span class="k">def</span> <span class="nf">stage_to_result</span><span class="p">(</span><span class="n">stage</span><span class="p">):</span> |
| <span class="k">return</span> <span class="p">{</span><span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">:</span> <span class="n">expr_to_pcoll</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span> |
| <span class="k">for</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">stage</span><span class="o">.</span><span class="n">inputs</span><span class="p">}</span> <span class="o">|</span> <span class="n">ComputeStage</span><span class="p">(</span><span class="n">stage</span><span class="p">)</span> |
| |
| <span class="nd">@_memoize</span> |
| <span class="k">def</span> <span class="nf">expr_to_pcoll</span><span class="p">(</span><span class="n">expr</span><span class="p">):</span> |
| <span class="k">if</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">inputs</span><span class="p">[</span><span class="n">expr</span><span class="p">]</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">stage_to_result</span><span class="p">(</span><span class="n">expr_to_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">))[</span><span class="n">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">]</span> |
| |
| <span class="nd">@_memoize</span> |
| <span class="k">def</span> <span class="nf">estimate_size</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">same_stage_ok</span><span class="p">):</span> |
| <span class="c1"># Returns a pcollection of ints whose sum is the estimated size of the</span> |
| <span class="c1"># given expression.</span> |
| <span class="n">pipeline</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="nb">iter</span><span class="p">(</span><span class="n">inputs</span><span class="o">.</span><span class="n">values</span><span class="p">()))</span><span class="o">.</span><span class="n">pipeline</span> |
| <span class="n">label</span> <span class="o">=</span> <span class="s1">'Size[</span><span class="si">%s</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">expr</span><span class="o">.</span><span class="n">_id</span><span class="p">,</span> <span class="n">same_stage_ok</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">is_scalar</span><span class="p">(</span><span class="n">expr</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">pipeline</span> <span class="o">|</span> <span class="n">label</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Create</span><span class="p">([</span><span class="mi">0</span><span class="p">])</span> |
| <span class="k">elif</span> <span class="n">same_stage_ok</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">expr_to_pcoll</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span> <span class="o">|</span> <span class="n">label</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Map</span><span class="p">(</span><span class="n">_total_memory_usage</span><span class="p">)</span> |
| <span class="k">elif</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">:</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># This is the stage to avoid.</span> |
| <span class="n">expr_stage</span> <span class="o">=</span> <span class="n">expr_to_stage</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span> |
| <span class="c1"># If the stage doesn't start with a shuffle, it's not safe to fuse</span> |
| <span class="c1"># the computation into its parent either.</span> |
| <span class="n">has_shuffle</span> <span class="o">=</span> <span class="n">expr_stage</span><span class="o">.</span><span class="n">partitioning</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"># We assume the size of an expression is the sum of the size of its</span> |
| <span class="c1"># inputs, which may be off by quite a bit, but the goal is to get</span> |
| <span class="c1"># within an order of magnitude or two.</span> |
| <span class="n">arg_sizes</span> <span class="o">=</span> <span class="p">[]</span> |
| <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">expr</span><span class="o">.</span><span class="n">args</span><span class="p">():</span> |
| <span class="k">if</span> <span class="n">is_scalar</span><span class="p">(</span><span class="n">arg</span><span class="p">):</span> |
| <span class="k">continue</span> |
| <span class="k">elif</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">:</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| <span class="n">arg_size</span> <span class="o">=</span> <span class="n">estimate_size</span><span class="p">(</span> |
| <span class="n">arg</span><span class="p">,</span> |
| <span class="n">same_stage_ok</span><span class="o">=</span><span class="n">has_shuffle</span> <span class="ow">and</span> <span class="n">expr_to_stage</span><span class="p">(</span><span class="n">arg</span><span class="p">)</span> <span class="o">!=</span> <span class="n">expr_stage</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">arg_size</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| <span class="n">arg_sizes</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">arg_size</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">arg_sizes</span> <span class="o">|</span> <span class="n">label</span> <span class="o">>></span> <span class="n">beam</span><span class="o">.</span><span class="n">Flatten</span><span class="p">(</span><span class="n">pipeline</span><span class="o">=</span><span class="n">pipeline</span><span class="p">)</span> |
| |
| <span class="c1"># Now we can compute and return the result.</span> |
| <span class="k">return</span> <span class="p">{</span><span class="n">k</span><span class="p">:</span> <span class="n">expr_to_pcoll</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span> <span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">expr</span> <span class="ow">in</span> <span class="n">outputs</span><span class="o">.</span><span class="n">items</span><span class="p">()}</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_total_memory_usage</span><span class="p">(</span><span class="n">frame</span><span class="p">):</span> |
| <span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">frame</span><span class="p">,</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">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">))</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">size</span> <span class="o">=</span> <span class="n">frame</span><span class="o">.</span><span class="n">memory_usage</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">size</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span> |
| <span class="n">size</span> <span class="o">=</span> <span class="n">size</span><span class="o">.</span><span class="n">sum</span><span class="p">()</span> <span class="o">+</span> <span class="n">PER_COL_OVERHEAD</span> <span class="o">*</span> <span class="nb">len</span><span class="p">(</span><span class="n">size</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">size</span> <span class="o">+=</span> <span class="n">PER_COL_OVERHEAD</span> |
| <span class="k">return</span> <span class="n">size</span> |
| <span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span> |
| <span class="c1"># Don't know, assume it's really big.</span> |
| <span class="nb">float</span><span class="p">(</span><span class="s1">'inf'</span><span class="p">)</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_PreBatch</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">target_size</span><span class="o">=</span><span class="n">TARGET_PARTITION_SIZE</span><span class="p">,</span> <span class="n">min_size</span><span class="o">=</span><span class="n">MIN_PARTITION_SIZE</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_size</span> <span class="o">=</span> <span class="n">target_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_min_size</span> <span class="o">=</span> <span class="n">min_size</span> |
| |
| <span class="k">def</span> <span class="nf">start_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_parts</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">list</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_running_size</span> <span class="o">=</span> <span class="mi">0</span> |
| |
| <span class="k">def</span> <span class="nf">process</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">part</span><span class="p">,</span> |
| <span class="n">window</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">,</span> |
| <span class="n">timestamp</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">):</span> |
| <span class="n">part_size</span> <span class="o">=</span> <span class="n">_total_memory_usage</span><span class="p">(</span><span class="n">part</span><span class="p">)</span> |
| <span class="k">if</span> <span class="n">part_size</span> <span class="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_min_size</span><span class="p">:</span> |
| <span class="k">yield</span> <span class="n">part</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_running_size</span> <span class="o">+=</span> <span class="n">part_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_parts</span><span class="p">[</span><span class="n">window</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">part</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_running_size</span> <span class="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_size</span><span class="p">:</span> |
| <span class="k">yield from</span> <span class="bp">self</span><span class="o">.</span><span class="n">finish_bundle</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">),</span> <span class="n">parts</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_parts</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="k">yield</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span><span class="n">_concat</span><span class="p">(</span><span class="n">parts</span><span class="p">),</span> <span class="n">timestamp</span><span class="p">,</span> <span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">start_bundle</span><span class="p">()</span> |
| |
| |
| <span class="k">class</span> <span class="nc">_ReBatch</span><span class="p">(</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""Groups all the parts from various workers into the same dataframe.</span> |
| |
| <span class="sd"> Also groups across partitions, up to a given data size, to recover some</span> |
| <span class="sd"> efficiency in the face of over-partitioning.</span> |
| <span class="sd"> """</span> |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> <span class="n">target_size</span><span class="o">=</span><span class="n">TARGET_PARTITION_SIZE</span><span class="p">,</span> <span class="n">min_size</span><span class="o">=</span><span class="n">MIN_PARTITION_SIZE</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_target_size</span> <span class="o">=</span> <span class="n">target_size</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_min_size</span> <span class="o">=</span> <span class="n">min_size</span> |
| |
| <span class="k">def</span> <span class="nf">start_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_parts</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="k">lambda</span><span class="p">:</span> <span class="n">collections</span><span class="o">.</span><span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_running_size</span> <span class="o">=</span> <span class="mi">0</span> |
| |
| <span class="k">def</span> <span class="nf">process</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">element</span><span class="p">,</span> |
| <span class="n">window</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">WindowParam</span><span class="p">,</span> |
| <span class="n">timestamp</span><span class="o">=</span><span class="n">beam</span><span class="o">.</span><span class="n">DoFn</span><span class="o">.</span><span class="n">TimestampParam</span><span class="p">):</span> |
| <span class="n">_</span><span class="p">,</span> <span class="n">tagged_parts</span> <span class="o">=</span> <span class="n">element</span> |
| <span class="k">for</span> <span class="n">tag</span><span class="p">,</span> <span class="n">parts</span> <span class="ow">in</span> <span class="n">tagged_parts</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="k">for</span> <span class="n">part</span> <span class="ow">in</span> <span class="n">parts</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_running_size</span> <span class="o">+=</span> <span class="n">_total_memory_usage</span><span class="p">(</span><span class="n">part</span><span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_parts</span><span class="p">[</span><span class="n">window</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">][</span><span class="n">tag</span><span class="p">]</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">parts</span><span class="p">)</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_running_size</span> <span class="o">>=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_target_size</span><span class="p">:</span> |
| <span class="k">yield from</span> <span class="bp">self</span><span class="o">.</span><span class="n">finish_bundle</span><span class="p">()</span> |
| |
| <span class="k">def</span> <span class="nf">finish_bundle</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">timestamp</span><span class="p">),</span> <span class="n">tagged_parts</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_parts</span><span class="o">.</span><span class="n">items</span><span class="p">():</span> |
| <span class="k">yield</span> <span class="n">windowed_value</span><span class="o">.</span><span class="n">WindowedValue</span><span class="p">(</span> <span class="c1"># yapf break</span> |
| <span class="p">{</span> |
| <span class="n">tag</span><span class="p">:</span> <span class="n">_concat</span><span class="p">(</span><span class="n">parts</span><span class="p">)</span> <span class="k">if</span> <span class="n">parts</span> <span class="k">else</span> <span class="kc">None</span> |
| <span class="k">for</span> <span class="p">(</span><span class="n">tag</span><span class="p">,</span> <span class="n">parts</span><span class="p">)</span> <span class="ow">in</span> <span class="n">tagged_parts</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="p">},</span> |
| <span class="n">timestamp</span><span class="p">,</span> <span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="p">))</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">start_bundle</span><span class="p">()</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_memoize</span><span class="p">(</span><span class="n">f</span><span class="p">):</span> |
| <span class="n">cache</span> <span class="o">=</span> <span class="p">{}</span> |
| |
| <span class="k">def</span> <span class="nf">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="n">key</span> <span class="o">=</span> <span class="n">args</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">(</span><span class="nb">sorted</span><span class="p">(</span><span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">()))</span> |
| <span class="k">if</span> <span class="n">key</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">cache</span><span class="p">:</span> |
| <span class="n">cache</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</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">return</span> <span class="n">cache</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> |
| |
| <span class="k">return</span> <span class="n">wrapper</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_dict_union</span><span class="p">(</span><span class="n">dicts</span><span class="p">):</span> |
| <span class="n">result</span> <span class="o">=</span> <span class="p">{}</span> |
| <span class="k">for</span> <span class="n">d</span> <span class="ow">in</span> <span class="n">dicts</span><span class="p">:</span> |
| <span class="n">result</span><span class="o">.</span><span class="n">update</span><span class="p">(</span><span class="n">d</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">result</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_concat</span><span class="p">(</span><span class="n">parts</span><span class="p">):</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">parts</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">parts</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">pd</span><span class="o">.</span><span class="n">concat</span><span class="p">(</span><span class="n">parts</span><span class="p">)</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_flatten</span><span class="p">(</span> |
| <span class="n">valueish</span><span class="p">,</span> <span class="c1"># type: Union[T, List[T], Tuple[T], Dict[Any, T]]</span> |
| <span class="n">root</span><span class="o">=</span><span class="p">(),</span> <span class="c1"># type: Tuple[Any, ...]</span> |
| <span class="p">):</span> |
| <span class="c1"># type: (...) -> Mapping[Tuple[Any, ...], T]</span> |
| |
| <span class="w"> </span><span class="sd">"""Given a nested structure of dicts, tuples, and lists, return a flat</span> |
| <span class="sd"> dictionary where the values are the leafs and the keys are the "paths" to</span> |
| <span class="sd"> these leaves.</span> |
| |
| <span class="sd"> For example `{a: x, b: (y, z)}` becomes `{(a,): x, (b, 0): y, (b, 1): c}`.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">valueish</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span> |
| <span class="k">return</span> <span class="n">_dict_union</span><span class="p">(</span><span class="n">_flatten</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">root</span> <span class="o">+</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="p">))</span> <span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">valueish</span><span class="o">.</span><span class="n">items</span><span class="p">())</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">valueish</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="k">return</span> <span class="n">_dict_union</span><span class="p">(</span> |
| <span class="n">_flatten</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">root</span> <span class="o">+</span> <span class="p">(</span><span class="n">ix</span><span class="p">,</span> <span class="p">))</span> <span class="k">for</span> <span class="n">ix</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">valueish</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">root</span><span class="p">:</span> <span class="n">valueish</span><span class="p">}</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_substitute</span><span class="p">(</span><span class="n">valueish</span><span class="p">,</span> <span class="n">replacements</span><span class="p">,</span> <span class="n">root</span><span class="o">=</span><span class="p">()):</span> |
| <span class="w"> </span><span class="sd">"""Substitutes the values in valueish with those in replacements where the</span> |
| <span class="sd"> keys are as in _flatten.</span> |
| |
| <span class="sd"> For example,</span> |
| |
| <span class="sd"> ```</span> |
| <span class="sd"> _substitute(</span> |
| <span class="sd"> {a: x, b: (y, z)},</span> |
| <span class="sd"> {(a,): X, (b, 0): Y, (b, 1): Z})</span> |
| <span class="sd"> ```</span> |
| |
| <span class="sd"> returns `{a: X, b: (Y, Z)}`.</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">valueish</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span> |
| <span class="k">return</span> <span class="nb">type</span><span class="p">(</span><span class="n">valueish</span><span class="p">)({</span> |
| <span class="n">k</span><span class="p">:</span> <span class="n">_substitute</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">replacements</span><span class="p">,</span> <span class="n">root</span> <span class="o">+</span> <span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="p">))</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">valueish</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="p">})</span> |
| <span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">valueish</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="k">return</span> <span class="nb">type</span><span class="p">(</span><span class="n">valueish</span><span class="p">)((</span> |
| <span class="n">_substitute</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">replacements</span><span class="p">,</span> <span class="n">root</span> <span class="o">+</span> <span class="p">(</span><span class="n">ix</span><span class="p">,</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">v</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">valueish</span><span class="p">)))</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="k">return</span> <span class="n">replacements</span><span class="p">[</span><span class="n">root</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> |