blob: 3836c028b94792f4f4f818ef3a8830c37fbe93f4 [file] [log] [blame]
<!DOCTYPE html>
<!--[if IE 8]><html class="no-js lt-ie9" lang="en" > <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js" lang="en" > <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta name="viewport" content="width=device-width, initial-scale=1.0">
<title>apache_beam.dataframe.doctests &mdash; 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> &raquo;</li>
<li><a href="../../index.html">Module code</a> &raquo;</li>
<li>apache_beam.dataframe.doctests</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.doctests</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="sd">&quot;&quot;&quot;A module that allows running existing pandas doctests with Beam dataframes.</span>
<span class="sd">This module hooks into the doctesting framework by providing a custom</span>
<span class="sd">runner and, in particular, an OutputChecker, as well as providing a fake</span>
<span class="sd">object for mocking out the pandas module.</span>
<span class="sd">The (novel) sequence of events when running a doctest is as follows.</span>
<span class="sd"> 1. The test invokes `pd.DataFrame(...)` (or similar) and an actual dataframe</span>
<span class="sd"> is computed and stashed but a Beam deferred dataframe is returned</span>
<span class="sd"> in its place.</span>
<span class="sd"> 2. Computations are done on these &quot;dataframes,&quot; resulting in new objects,</span>
<span class="sd"> but as these are actually deferred, only expression trees are built.</span>
<span class="sd"> In the background, a mapping of id -&gt; deferred dataframe is stored for</span>
<span class="sd"> each newly created dataframe.</span>
<span class="sd"> 3. When any dataframe is printed out, the repr has been overwritten to</span>
<span class="sd"> print `Dataframe[id]`. The aforementened mapping is used to map this back</span>
<span class="sd"> to the actual dataframe object, which is then computed via Beam, and its</span>
<span class="sd"> the (stringified) result plugged into the actual output for comparison.</span>
<span class="sd"> 4. The comparison is then done on the sorted lines of the expected and actual</span>
<span class="sd"> values.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="kn">import</span> <span class="nn">collections</span>
<span class="kn">import</span> <span class="nn">contextlib</span>
<span class="kn">import</span> <span class="nn">doctest</span>
<span class="kn">import</span> <span class="nn">re</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">import</span> <span class="nn">traceback</span>
<span class="kn">from</span> <span class="nn">io</span> <span class="kn">import</span> <span class="n">StringIO</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">import</span> <span class="nn">numpy</span> <span class="k">as</span> <span class="nn">np</span>
<span class="kn">import</span> <span class="nn">pandas</span> <span class="k">as</span> <span class="nn">pd</span>
<span class="kn">import</span> <span class="nn">apache_beam</span> <span class="k">as</span> <span class="nn">beam</span>
<span class="kn">from</span> <span class="nn">apache_beam.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">pandas_top_level_functions</span>
<span class="kn">from</span> <span class="nn">apache_beam.dataframe</span> <span class="kn">import</span> <span class="n">transforms</span>
<span class="kn">from</span> <span class="nn">apache_beam.dataframe.frame_base</span> <span class="kn">import</span> <span class="n">DeferredBase</span>
<div class="viewcode-block" id="FakePandasObject"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.FakePandasObject">[docs]</a><span class="k">class</span> <span class="nc">FakePandasObject</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A stand-in for the wrapped pandas objects.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pandas_obj</span><span class="p">,</span> <span class="n">test_env</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_pandas_obj</span> <span class="o">=</span> <span class="n">pandas_obj</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_test_env</span> <span class="o">=</span> <span class="n">test_env</span>
<span class="k">def</span> <span class="fm">__call__</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">result</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pandas_obj</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">type</span><span class="p">(</span><span class="n">result</span><span class="p">)</span> <span class="ow">in</span> <span class="n">DeferredBase</span><span class="o">.</span><span class="n">_pandas_type_map</span><span class="p">:</span>
<span class="n">placeholder</span> <span class="o">=</span> <span class="n">expressions</span><span class="o">.</span><span class="n">PlaceholderExpression</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="mi">0</span><span class="p">:</span><span class="mi">0</span><span class="p">])</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_test_env</span><span class="o">.</span><span class="n">_inputs</span><span class="p">[</span><span class="n">placeholder</span><span class="p">]</span> <span class="o">=</span> <span class="n">result</span>
<span class="k">return</span> <span class="n">DeferredBase</span><span class="o">.</span><span class="n">wrap</span><span class="p">(</span><span class="n">placeholder</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="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="n">attr</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_pandas_obj</span><span class="p">,</span> <span class="n">name</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">callable</span><span class="p">(</span><span class="n">attr</span><span class="p">):</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">FakePandasObject</span><span class="p">(</span><span class="n">attr</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_test_env</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">attr</span>
<span class="c1"># Cache this so two lookups return the same object.</span>
<span class="nb">setattr</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">result</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span>
<span class="k">def</span> <span class="nf">__reduce__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="k">lambda</span><span class="p">:</span> <span class="n">pd</span><span class="p">,</span> <span class="p">()</span></div>
<div class="viewcode-block" id="TestEnvironment"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.TestEnvironment">[docs]</a><span class="k">class</span> <span class="nc">TestEnvironment</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A class managing the patching (of methods, inputs, and outputs) needed</span>
<span class="sd"> to run and validate tests.</span>
<span class="sd"> These classes are patched to be able to recognize and retrieve inputs</span>
<span class="sd"> and results, stored in `self._inputs` and `self._all_frames` respectively.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_inputs</span> <span class="o">=</span> <span class="p">{}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_all_frames</span> <span class="o">=</span> <span class="p">{}</span>
<div class="viewcode-block" id="TestEnvironment.fake_pandas_module"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.TestEnvironment.fake_pandas_module">[docs]</a> <span class="k">def</span> <span class="nf">fake_pandas_module</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">FakePandasObject</span><span class="p">(</span><span class="n">pandas_top_level_functions</span><span class="o">.</span><span class="n">pd_wrapper</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span></div>
<span class="nd">@contextlib</span><span class="o">.</span><span class="n">contextmanager</span>
<span class="k">def</span> <span class="nf">_monkey_patch_type</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">deferred_type</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Monkey-patch __init__ to record a pointer to all created frames, and</span>
<span class="sd"> __repr__ to be able to recognize them in the doctest output.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">old_init</span><span class="p">,</span> <span class="n">old_repr</span> <span class="o">=</span> <span class="n">deferred_type</span><span class="o">.</span><span class="fm">__init__</span><span class="p">,</span> <span class="n">deferred_type</span><span class="o">.</span><span class="fm">__repr__</span>
<span class="k">def</span> <span class="nf">new_init</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="n">old_init</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="bp">self</span><span class="o">.</span><span class="n">_all_frames</span><span class="p">[</span><span class="nb">id</span><span class="p">(</span><span class="n">df</span><span class="p">)]</span> <span class="o">=</span> <span class="n">df</span>
<span class="n">deferred_type</span><span class="o">.</span><span class="fm">__init__</span> <span class="o">=</span> <span class="n">new_init</span>
<span class="n">deferred_type</span><span class="o">.</span><span class="fm">__repr__</span> <span class="o">=</span> <span class="k">lambda</span> <span class="bp">self</span><span class="p">:</span> <span class="s1">&#39;DeferredBase[</span><span class="si">%s</span><span class="s1">]&#39;</span> <span class="o">%</span> <span class="nb">id</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">_recorded_results</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="k">yield</span>
<span class="k">finally</span><span class="p">:</span>
<span class="n">deferred_type</span><span class="o">.</span><span class="fm">__init__</span><span class="p">,</span> <span class="n">deferred_type</span><span class="o">.</span><span class="fm">__repr__</span> <span class="o">=</span> <span class="n">old_init</span><span class="p">,</span> <span class="n">old_repr</span>
<div class="viewcode-block" id="TestEnvironment.context"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.TestEnvironment.context">[docs]</a> <span class="nd">@contextlib</span><span class="o">.</span><span class="n">contextmanager</span>
<span class="k">def</span> <span class="nf">context</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Creates a context within which DeferredBase types are monkey patched</span>
<span class="sd"> to record ids.&quot;&quot;&quot;</span>
<span class="k">with</span> <span class="n">contextlib</span><span class="o">.</span><span class="n">ExitStack</span><span class="p">()</span> <span class="k">as</span> <span class="n">stack</span><span class="p">:</span>
<span class="k">for</span> <span class="n">deferred_type</span> <span class="ow">in</span> <span class="n">DeferredBase</span><span class="o">.</span><span class="n">_pandas_type_map</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="n">stack</span><span class="o">.</span><span class="n">enter_context</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_monkey_patch_type</span><span class="p">(</span><span class="n">deferred_type</span><span class="p">))</span>
<span class="k">yield</span></div></div>
<span class="k">class</span> <span class="nc">_InMemoryResultRecorder</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Helper for extracting computed results from a Beam pipeline.</span>
<span class="sd"> Used as follows::</span>
<span class="sd"> with _InMemoryResultRecorder() as recorder:</span>
<span class="sd"> with beam.Pipeline() as p:</span>
<span class="sd"> ...</span>
<span class="sd"> pcoll | beam.Map(recorder.record_fn(name))</span>
<span class="sd"> seen = recorder.get_recorded(name)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Class-level value to survive pickling.</span>
<span class="n">_ALL_RESULTS</span> <span class="o">=</span> <span class="p">{}</span> <span class="c1"># type: Dict[str, List[Any]]</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="bp">self</span><span class="o">.</span><span class="n">_id</span> <span class="o">=</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="fm">__enter__</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">_ALL_RESULTS</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_id</span><span class="p">]</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="k">return</span> <span class="bp">self</span>
<span class="k">def</span> <span class="fm">__exit__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">unused_args</span><span class="p">):</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_ALL_RESULTS</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_id</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">record_fn</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="nf">record</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">_ALL_RESULTS</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_id</span><span class="p">][</span><span class="n">name</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">return</span> <span class="n">record</span>
<span class="k">def</span> <span class="nf">get_recorded</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="bp">self</span><span class="o">.</span><span class="n">_ALL_RESULTS</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_id</span><span class="p">][</span><span class="n">name</span><span class="p">]</span>
<span class="n">WONT_IMPLEMENT</span> <span class="o">=</span> <span class="s1">&#39;apache_beam.dataframe.frame_base.WontImplementError&#39;</span>
<span class="n">NOT_IMPLEMENTED</span> <span class="o">=</span> <span class="s1">&#39;NotImplementedError&#39;</span>
<span class="k">class</span> <span class="nc">_DeferrredDataframeOutputChecker</span><span class="p">(</span><span class="n">doctest</span><span class="o">.</span><span class="n">OutputChecker</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Validates output by replacing DeferredBase[...] with computed values.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">env</span><span class="p">,</span> <span class="n">use_beam</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_env</span> <span class="o">=</span> <span class="n">env</span>
<span class="k">if</span> <span class="n">use_beam</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">compute</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">compute_using_beam</span>
<span class="k">else</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">compute</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">compute_using_session</span>
<span class="bp">self</span><span class="o">.</span><span class="n">reset</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">reset</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_last_error</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">compute_using_session</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_compute</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">PartitioningSession</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_env</span><span class="o">.</span><span class="n">_inputs</span><span class="p">)</span>
<span class="k">return</span> <span class="p">{</span>
<span class="n">name</span><span class="p">:</span> <span class="n">session</span><span class="o">.</span><span class="n">evaluate</span><span class="p">(</span><span class="n">frame</span><span class="o">.</span><span class="n">_expr</span><span class="p">)</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span>
<span class="n">frame</span> <span class="ow">in</span> <span class="n">to_compute</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span>
<span class="k">def</span> <span class="nf">compute_using_beam</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">to_compute</span><span class="p">):</span>
<span class="k">with</span> <span class="n">_InMemoryResultRecorder</span><span class="p">()</span> <span class="k">as</span> <span class="n">recorder</span><span class="p">:</span>
<span class="k">with</span> <span class="n">beam</span><span class="o">.</span><span class="n">Pipeline</span><span class="p">()</span> <span class="k">as</span> <span class="n">p</span><span class="p">:</span>
<span class="n">input_pcolls</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">placeholder</span><span class="p">:</span> <span class="n">p</span>
<span class="o">|</span> <span class="s1">&#39;Create</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">placeholder</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">Create</span><span class="p">([</span><span class="nb">input</span><span class="p">[::</span><span class="mi">2</span><span class="p">],</span> <span class="nb">input</span><span class="p">[</span><span class="mi">1</span><span class="p">::</span><span class="mi">2</span><span class="p">]])</span>
<span class="k">for</span> <span class="n">placeholder</span><span class="p">,</span>
<span class="nb">input</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_env</span><span class="o">.</span><span class="n">_inputs</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span>
<span class="n">output_pcolls</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">input_pcolls</span> <span class="o">|</span> <span class="n">transforms</span><span class="o">.</span><span class="n">_DataframeExpressionsTransform</span><span class="p">(</span>
<span class="p">{</span><span class="n">name</span><span class="p">:</span> <span class="n">frame</span><span class="o">.</span><span class="n">_expr</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">frame</span> <span class="ow">in</span> <span class="n">to_compute</span><span class="o">.</span><span class="n">items</span><span class="p">()}))</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">output_pcoll</span> <span class="ow">in</span> <span class="n">output_pcolls</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">_</span> <span class="o">=</span> <span class="n">output_pcoll</span> <span class="o">|</span> <span class="s1">&#39;Record</span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">name</span> <span class="o">&gt;&gt;</span> <span class="n">beam</span><span class="o">.</span><span class="n">FlatMap</span><span class="p">(</span>
<span class="n">recorder</span><span class="o">.</span><span class="n">record_fn</span><span class="p">(</span><span class="n">name</span><span class="p">))</span>
<span class="c1"># pipeline runs, side effects recorded</span>
<span class="k">def</span> <span class="nf">concat</span><span class="p">(</span><span class="n">values</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">values</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</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">values</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">values</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">return</span> <span class="p">{</span>
<span class="n">name</span><span class="p">:</span> <span class="n">concat</span><span class="p">(</span><span class="n">recorder</span><span class="o">.</span><span class="n">get_recorded</span><span class="p">(</span><span class="n">name</span><span class="p">))</span>
<span class="k">for</span> <span class="n">name</span> <span class="ow">in</span> <span class="n">to_compute</span><span class="o">.</span><span class="n">keys</span><span class="p">()</span>
<span class="p">}</span>
<span class="k">def</span> <span class="nf">fix</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">want</span><span class="p">,</span> <span class="n">got</span><span class="p">):</span>
<span class="k">if</span> <span class="s1">&#39;DeferredBase&#39;</span> <span class="ow">in</span> <span class="n">got</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">to_compute</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="mi">0</span><span class="p">):</span> <span class="bp">self</span><span class="o">.</span><span class="n">_env</span><span class="o">.</span><span class="n">_all_frames</span><span class="p">[</span><span class="nb">int</span><span class="p">(</span><span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="mi">1</span><span class="p">))]</span>
<span class="k">for</span> <span class="n">m</span> <span class="ow">in</span> <span class="n">re</span><span class="o">.</span><span class="n">finditer</span><span class="p">(</span><span class="sa">r</span><span class="s1">&#39;DeferredBase\[(\d+)\]&#39;</span><span class="p">,</span> <span class="n">got</span><span class="p">)</span>
<span class="p">}</span>
<span class="n">computed</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">compute</span><span class="p">(</span><span class="n">to_compute</span><span class="p">)</span>
<span class="k">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">frame</span> <span class="ow">in</span> <span class="n">computed</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">got</span> <span class="o">=</span> <span class="n">got</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="nb">repr</span><span class="p">(</span><span class="n">frame</span><span class="p">))</span>
<span class="c1"># If a multiindex is used, compensate for it</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">frame</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">frame</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span> <span class="o">&gt;</span> <span class="mi">1</span> <span class="k">for</span> <span class="n">frame</span> <span class="ow">in</span> <span class="n">computed</span><span class="o">.</span><span class="n">values</span><span class="p">()):</span>
<span class="k">def</span> <span class="nf">fill_multiindex</span><span class="p">(</span><span class="n">text</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;An awful hack to work around the fact that pandas omits repeated</span>
<span class="sd"> elements in a multi-index.</span>
<span class="sd"> For example:</span>
<span class="sd"> Series name Row ID</span>
<span class="sd"> s1 0 a</span>
<span class="sd"> 1 b</span>
<span class="sd"> s2 0 c</span>
<span class="sd"> 1 d</span>
<span class="sd"> dtype: object</span>
<span class="sd"> The s1 and s2 are implied for the 2nd and 4th rows. However if we</span>
<span class="sd"> re-order this Series it might be printed this way:</span>
<span class="sd"> Series name Row ID</span>
<span class="sd"> s1 0 a</span>
<span class="sd"> s2 1 d</span>
<span class="sd"> s2 0 c</span>
<span class="sd"> s1 1 b</span>
<span class="sd"> dtype: object</span>
<span class="sd"> In our model these are equivalent, but when we sort the lines and</span>
<span class="sd"> check equality they are not. This method fills in any omitted</span>
<span class="sd"> multiindex values, so that we can successfully sort and compare.&quot;&quot;&quot;</span>
<span class="n">lines</span> <span class="o">=</span> <span class="p">[</span><span class="nb">list</span><span class="p">(</span><span class="n">line</span><span class="p">)</span> <span class="k">for</span> <span class="n">line</span> <span class="ow">in</span> <span class="n">text</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">)]</span>
<span class="k">for</span> <span class="n">prev</span><span class="p">,</span> <span class="n">line</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">lines</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">],</span> <span class="n">lines</span><span class="p">[</span><span class="mi">1</span><span class="p">:]):</span>
<span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="n">l</span> <span class="o">==</span> <span class="s1">&#39; &#39;</span> <span class="k">for</span> <span class="n">l</span> <span class="ow">in</span> <span class="n">line</span><span class="p">):</span>
<span class="k">continue</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">l</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">line</span><span class="p">):</span>
<span class="k">if</span> <span class="n">l</span> <span class="o">!=</span> <span class="s1">&#39; &#39;</span><span class="p">:</span>
<span class="k">break</span>
<span class="n">line</span><span class="p">[</span><span class="n">i</span><span class="p">]</span> <span class="o">=</span> <span class="n">prev</span><span class="p">[</span><span class="n">i</span><span class="p">]</span>
<span class="k">return</span> <span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="s1">&#39;&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">line</span><span class="p">)</span> <span class="k">for</span> <span class="n">line</span> <span class="ow">in</span> <span class="n">lines</span><span class="p">)</span>
<span class="n">got</span> <span class="o">=</span> <span class="n">fill_multiindex</span><span class="p">(</span><span class="n">got</span><span class="p">)</span>
<span class="n">want</span> <span class="o">=</span> <span class="n">fill_multiindex</span><span class="p">(</span><span class="n">want</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">sort_and_normalize</span><span class="p">(</span><span class="n">text</span><span class="p">):</span>
<span class="k">return</span> <span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span>
<span class="nb">sorted</span><span class="p">(</span>
<span class="p">[</span><span class="n">line</span><span class="o">.</span><span class="n">rstrip</span><span class="p">()</span> <span class="k">for</span> <span class="n">line</span> <span class="ow">in</span> <span class="n">text</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">)</span> <span class="k">if</span> <span class="n">line</span><span class="o">.</span><span class="n">strip</span><span class="p">()],</span>
<span class="n">key</span><span class="o">=</span><span class="nb">str</span><span class="o">.</span><span class="n">strip</span><span class="p">))</span> <span class="o">+</span> <span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span>
<span class="n">got</span> <span class="o">=</span> <span class="n">sort_and_normalize</span><span class="p">(</span><span class="n">got</span><span class="p">)</span>
<span class="n">want</span> <span class="o">=</span> <span class="n">sort_and_normalize</span><span class="p">(</span><span class="n">want</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">Exception</span><span class="p">:</span>
<span class="n">got</span> <span class="o">=</span> <span class="n">traceback</span><span class="o">.</span><span class="n">format_exc</span><span class="p">()</span>
<span class="k">return</span> <span class="n">want</span><span class="p">,</span> <span class="n">got</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_seen_error</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">_last_error</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">check_output</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">want</span><span class="p">,</span> <span class="n">got</span><span class="p">,</span> <span class="n">optionflags</span><span class="p">):</span>
<span class="c1"># When an error occurs check_output is called with want=example.exc_msg,</span>
<span class="c1"># and got=exc_msg</span>
<span class="c1"># First check if `want` is a special string indicating wont_implement_ok</span>
<span class="c1"># and/or not_implemented_ok</span>
<span class="n">allowed_exceptions</span> <span class="o">=</span> <span class="n">want</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;|&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="n">exc</span> <span class="ow">in</span> <span class="p">(</span><span class="n">WONT_IMPLEMENT</span><span class="p">,</span> <span class="n">NOT_IMPLEMENTED</span><span class="p">)</span>
<span class="k">for</span> <span class="n">exc</span> <span class="ow">in</span> <span class="n">allowed_exceptions</span><span class="p">):</span>
<span class="c1"># If it is, check for WontImplementError and NotImplementedError</span>
<span class="k">if</span> <span class="n">WONT_IMPLEMENT</span> <span class="ow">in</span> <span class="n">allowed_exceptions</span> <span class="ow">and</span> <span class="n">got</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span>
<span class="n">WONT_IMPLEMENT</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_last_error</span> <span class="o">=</span> <span class="n">WONT_IMPLEMENT</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">elif</span> <span class="n">NOT_IMPLEMENTED</span> <span class="ow">in</span> <span class="n">allowed_exceptions</span> <span class="ow">and</span> <span class="n">got</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span>
<span class="n">NOT_IMPLEMENTED</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_last_error</span> <span class="o">=</span> <span class="n">NOT_IMPLEMENTED</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="k">elif</span> <span class="n">got</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">&#39;NameError&#39;</span><span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_seen_error</span><span class="p">:</span>
<span class="c1"># This allows us to gracefully skip tests like</span>
<span class="c1"># &gt;&gt;&gt; res = df.unsupported_operation()</span>
<span class="c1"># &gt;&gt;&gt; check(res)</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="bp">self</span><span class="o">.</span><span class="n">reset</span><span class="p">()</span>
<span class="n">want</span><span class="p">,</span> <span class="n">got</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">fix</span><span class="p">(</span><span class="n">want</span><span class="p">,</span> <span class="n">got</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">check_output</span><span class="p">(</span><span class="n">want</span><span class="p">,</span> <span class="n">got</span><span class="p">,</span> <span class="n">optionflags</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">output_difference</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">example</span><span class="p">,</span> <span class="n">got</span><span class="p">,</span> <span class="n">optionflags</span><span class="p">):</span>
<span class="n">want</span><span class="p">,</span> <span class="n">got</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">fix</span><span class="p">(</span><span class="n">example</span><span class="o">.</span><span class="n">want</span><span class="p">,</span> <span class="n">got</span><span class="p">)</span>
<span class="k">if</span> <span class="n">want</span> <span class="o">!=</span> <span class="n">example</span><span class="o">.</span><span class="n">want</span><span class="p">:</span>
<span class="n">example</span> <span class="o">=</span> <span class="n">doctest</span><span class="o">.</span><span class="n">Example</span><span class="p">(</span>
<span class="n">example</span><span class="o">.</span><span class="n">source</span><span class="p">,</span>
<span class="n">want</span><span class="p">,</span>
<span class="n">example</span><span class="o">.</span><span class="n">exc_msg</span><span class="p">,</span>
<span class="n">example</span><span class="o">.</span><span class="n">lineno</span><span class="p">,</span>
<span class="n">example</span><span class="o">.</span><span class="n">indent</span><span class="p">,</span>
<span class="n">example</span><span class="o">.</span><span class="n">options</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">output_difference</span><span class="p">(</span><span class="n">example</span><span class="p">,</span> <span class="n">got</span><span class="p">,</span> <span class="n">optionflags</span><span class="p">)</span>
<div class="viewcode-block" id="BeamDataframeDoctestRunner"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.BeamDataframeDoctestRunner">[docs]</a><span class="k">class</span> <span class="nc">BeamDataframeDoctestRunner</span><span class="p">(</span><span class="n">doctest</span><span class="o">.</span><span class="n">DocTestRunner</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;A Doctest runner suitable for replacing the `pd` module with one backed</span>
<span class="sd"> by beam.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">env</span><span class="p">,</span>
<span class="n">use_beam</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="n">wont_implement_ok</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">not_implemented_ok</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">skip</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_test_env</span> <span class="o">=</span> <span class="n">env</span>
<span class="k">def</span> <span class="nf">to_callable</span><span class="p">(</span><span class="n">cond</span><span class="p">):</span>
<span class="k">if</span> <span class="n">cond</span> <span class="o">==</span> <span class="s1">&#39;*&#39;</span><span class="p">:</span>
<span class="k">return</span> <span class="k">lambda</span> <span class="n">example</span><span class="p">:</span> <span class="kc">True</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="k">lambda</span> <span class="n">example</span><span class="p">:</span> <span class="n">example</span><span class="o">.</span><span class="n">source</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span> <span class="o">==</span> <span class="n">cond</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_wont_implement_ok</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">test</span><span class="p">:</span> <span class="p">[</span><span class="n">to_callable</span><span class="p">(</span><span class="n">cond</span><span class="p">)</span> <span class="k">for</span> <span class="n">cond</span> <span class="ow">in</span> <span class="n">examples</span><span class="p">]</span>
<span class="k">for</span> <span class="n">test</span><span class="p">,</span>
<span class="n">examples</span> <span class="ow">in</span> <span class="p">(</span><span class="n">wont_implement_ok</span> <span class="ow">or</span> <span class="p">{})</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_not_implemented_ok</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">test</span><span class="p">:</span> <span class="p">[</span><span class="n">to_callable</span><span class="p">(</span><span class="n">cond</span><span class="p">)</span> <span class="k">for</span> <span class="n">cond</span> <span class="ow">in</span> <span class="n">examples</span><span class="p">]</span>
<span class="k">for</span> <span class="n">test</span><span class="p">,</span>
<span class="n">examples</span> <span class="ow">in</span> <span class="p">(</span><span class="n">not_implemented_ok</span> <span class="ow">or</span> <span class="p">{})</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_skip</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">test</span><span class="p">:</span> <span class="p">[</span><span class="n">to_callable</span><span class="p">(</span><span class="n">cond</span><span class="p">)</span> <span class="k">for</span> <span class="n">cond</span> <span class="ow">in</span> <span class="n">examples</span><span class="p">]</span>
<span class="k">for</span> <span class="n">test</span><span class="p">,</span>
<span class="n">examples</span> <span class="ow">in</span> <span class="p">(</span><span class="n">skip</span> <span class="ow">or</span> <span class="p">{})</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__init__</span><span class="p">(</span>
<span class="n">checker</span><span class="o">=</span><span class="n">_DeferrredDataframeOutputChecker</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_test_env</span><span class="p">,</span> <span class="n">use_beam</span><span class="p">),</span>
<span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">success</span> <span class="o">=</span> <span class="mi">0</span>
<span class="bp">self</span><span class="o">.</span><span class="n">skipped</span> <span class="o">=</span> <span class="mi">0</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_reasons</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">_skipped_set</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_is_wont_implement_ok</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">example</span><span class="p">,</span> <span class="n">test</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">any</span><span class="p">(</span>
<span class="n">wont_implement</span><span class="p">(</span><span class="n">example</span><span class="p">)</span>
<span class="k">for</span> <span class="n">wont_implement</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_wont_implement_ok</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">test</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> <span class="p">[]))</span>
<span class="k">def</span> <span class="nf">_is_not_implemented_ok</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">example</span><span class="p">,</span> <span class="n">test</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">any</span><span class="p">(</span>
<span class="n">not_implemented</span><span class="p">(</span><span class="n">example</span><span class="p">)</span>
<span class="k">for</span> <span class="n">not_implemented</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_not_implemented_ok</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">test</span><span class="o">.</span><span class="n">name</span><span class="p">,</span> <span class="p">[]))</span>
<div class="viewcode-block" id="BeamDataframeDoctestRunner.run"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.BeamDataframeDoctestRunner.run">[docs]</a> <span class="k">def</span> <span class="nf">run</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">test</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_checker</span><span class="o">.</span><span class="n">reset</span><span class="p">()</span>
<span class="k">for</span> <span class="n">example</span> <span class="ow">in</span> <span class="n">test</span><span class="o">.</span><span class="n">examples</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="n">should_skip</span><span class="p">(</span><span class="n">example</span><span class="p">)</span>
<span class="k">for</span> <span class="n">should_skip</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_skip</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">test</span><span class="o">.</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">_skipped_set</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">example</span><span class="p">)</span>
<span class="n">example</span><span class="o">.</span><span class="n">source</span> <span class="o">=</span> <span class="s1">&#39;pass&#39;</span>
<span class="n">example</span><span class="o">.</span><span class="n">want</span> <span class="o">=</span> <span class="s1">&#39;&#39;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">skipped</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="k">elif</span> <span class="n">example</span><span class="o">.</span><span class="n">exc_msg</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">allowed_exceptions</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_not_implemented_ok</span><span class="p">(</span><span class="n">example</span><span class="p">,</span> <span class="n">test</span><span class="p">):</span>
<span class="n">allowed_exceptions</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">NOT_IMPLEMENTED</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_is_wont_implement_ok</span><span class="p">(</span><span class="n">example</span><span class="p">,</span> <span class="n">test</span><span class="p">):</span>
<span class="n">allowed_exceptions</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">WONT_IMPLEMENT</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">allowed_exceptions</span><span class="p">):</span>
<span class="c1"># Don&#39;t fail doctests that raise this error.</span>
<span class="n">example</span><span class="o">.</span><span class="n">exc_msg</span> <span class="o">=</span> <span class="s1">&#39;|&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">allowed_exceptions</span><span class="p">)</span>
<span class="k">with</span> <span class="bp">self</span><span class="o">.</span><span class="n">_test_env</span><span class="o">.</span><span class="n">context</span><span class="p">():</span>
<span class="n">result</span> <span class="o">=</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">run</span><span class="p">(</span><span class="n">test</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="c1"># Can&#39;t add attributes to builtin result.</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">AugmentedTestResults</span><span class="p">(</span><span class="n">result</span><span class="o">.</span><span class="n">failed</span><span class="p">,</span> <span class="n">result</span><span class="o">.</span><span class="n">attempted</span><span class="p">)</span>
<span class="n">result</span><span class="o">.</span><span class="n">summary</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">summary</span><span class="p">()</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="BeamDataframeDoctestRunner.report_success"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.BeamDataframeDoctestRunner.report_success">[docs]</a> <span class="k">def</span> <span class="nf">report_success</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">out</span><span class="p">,</span> <span class="n">test</span><span class="p">,</span> <span class="n">example</span><span class="p">,</span> <span class="n">got</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">extract_concise_reason</span><span class="p">(</span><span class="n">got</span><span class="p">,</span> <span class="n">expected_exc</span><span class="p">):</span>
<span class="n">m</span> <span class="o">=</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="s2">&quot;Implement(?:ed)?Error:\s+(.*)\n$&quot;</span><span class="p">,</span> <span class="n">got</span><span class="p">)</span>
<span class="k">if</span> <span class="n">m</span><span class="p">:</span>
<span class="k">return</span> <span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span>
<span class="k">elif</span> <span class="s2">&quot;NameError&quot;</span> <span class="ow">in</span> <span class="n">got</span><span class="p">:</span>
<span class="k">return</span> <span class="s2">&quot;NameError following </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="n">expected_exc</span>
<span class="k">elif</span> <span class="n">re</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="sa">r</span><span class="s2">&quot;DeferredBase\[\d+\]\n&quot;</span><span class="p">,</span> <span class="n">got</span><span class="p">):</span>
<span class="k">return</span> <span class="s2">&quot;DeferredBase[*]&quot;</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">got</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span><span class="s2">&quot;</span><span class="se">\n</span><span class="s2">&quot;</span><span class="p">,</span> <span class="s2">&quot;</span><span class="se">\\</span><span class="s2">n&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_checker</span><span class="o">.</span><span class="n">_last_error</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_reasons</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_checker</span><span class="o">.</span><span class="n">_last_error</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">extract_concise_reason</span><span class="p">(</span><span class="n">got</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_checker</span><span class="o">.</span><span class="n">_last_error</span><span class="p">))</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_checker</span><span class="o">.</span><span class="n">_seen_error</span><span class="p">:</span>
<span class="n">m</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">search</span><span class="p">(</span><span class="s1">&#39;^([a-zA-Z0-9_, ]+)=&#39;</span><span class="p">,</span> <span class="n">example</span><span class="o">.</span><span class="n">source</span><span class="p">)</span>
<span class="k">if</span> <span class="n">m</span><span class="p">:</span>
<span class="k">for</span> <span class="n">var</span> <span class="ow">in</span> <span class="n">m</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;,&#39;</span><span class="p">):</span>
<span class="n">var</span> <span class="o">=</span> <span class="n">var</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span>
<span class="k">if</span> <span class="n">var</span> <span class="ow">in</span> <span class="n">test</span><span class="o">.</span><span class="n">globs</span><span class="p">:</span>
<span class="c1"># More informative to get a NameError than</span>
<span class="c1"># use the wrong previous value.</span>
<span class="k">del</span> <span class="n">test</span><span class="o">.</span><span class="n">globs</span><span class="p">[</span><span class="n">var</span><span class="p">]</span>
<span class="k">return</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">report_success</span><span class="p">(</span><span class="n">out</span><span class="p">,</span> <span class="n">test</span><span class="p">,</span> <span class="n">example</span><span class="p">,</span> <span class="n">got</span><span class="p">)</span></div>
<div class="viewcode-block" id="BeamDataframeDoctestRunner.fake_pandas_module"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.BeamDataframeDoctestRunner.fake_pandas_module">[docs]</a> <span class="k">def</span> <span class="nf">fake_pandas_module</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">_test_env</span><span class="o">.</span><span class="n">fake_pandas_module</span><span class="p">()</span></div>
<div class="viewcode-block" id="BeamDataframeDoctestRunner.summarize"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.BeamDataframeDoctestRunner.summarize">[docs]</a> <span class="k">def</span> <span class="nf">summarize</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">summarize</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">summary</span><span class="p">()</span><span class="o">.</span><span class="n">summarize</span><span class="p">()</span></div>
<div class="viewcode-block" id="BeamDataframeDoctestRunner.summary"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.BeamDataframeDoctestRunner.summary">[docs]</a> <span class="k">def</span> <span class="nf">summary</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">Summary</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">failures</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">tries</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">skipped</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reasons</span><span class="p">)</span></div></div>
<div class="viewcode-block" id="AugmentedTestResults"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.AugmentedTestResults">[docs]</a><span class="k">class</span> <span class="nc">AugmentedTestResults</span><span class="p">(</span><span class="n">doctest</span><span class="o">.</span><span class="n">TestResults</span><span class="p">):</span>
<span class="k">pass</span></div>
<div class="viewcode-block" id="Summary"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.Summary">[docs]</a><span class="k">class</span> <span class="nc">Summary</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">failures</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">tries</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">skipped</span><span class="o">=</span><span class="mi">0</span><span class="p">,</span> <span class="n">error_reasons</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">failures</span> <span class="o">=</span> <span class="n">failures</span>
<span class="bp">self</span><span class="o">.</span><span class="n">tries</span> <span class="o">=</span> <span class="n">tries</span>
<span class="bp">self</span><span class="o">.</span><span class="n">skipped</span> <span class="o">=</span> <span class="n">skipped</span>
<span class="bp">self</span><span class="o">.</span><span class="n">error_reasons</span> <span class="o">=</span> <span class="n">error_reasons</span> <span class="ow">or</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>
<div class="viewcode-block" id="Summary.result"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.Summary.result">[docs]</a> <span class="k">def</span> <span class="nf">result</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">res</span> <span class="o">=</span> <span class="n">AugmentedTestResults</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">failures</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">tries</span><span class="p">)</span>
<span class="n">res</span><span class="o">.</span><span class="n">summary</span> <span class="o">=</span> <span class="bp">self</span>
<span class="k">return</span> <span class="n">res</span></div>
<span class="k">def</span> <span class="fm">__add__</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">merged_reasons</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">key</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">error_reasons</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="p">[])</span> <span class="o">+</span> <span class="n">other</span><span class="o">.</span><span class="n">error_reasons</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="p">[])</span>
<span class="k">for</span> <span class="n">key</span> <span class="ow">in</span> <span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">error_reasons</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span><span class="o">.</span><span class="n">union</span><span class="p">(</span>
<span class="n">other</span><span class="o">.</span><span class="n">error_reasons</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span>
<span class="p">}</span>
<span class="k">return</span> <span class="n">Summary</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">failures</span> <span class="o">+</span> <span class="n">other</span><span class="o">.</span><span class="n">failures</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">tries</span> <span class="o">+</span> <span class="n">other</span><span class="o">.</span><span class="n">tries</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">skipped</span> <span class="o">+</span> <span class="n">other</span><span class="o">.</span><span class="n">skipped</span><span class="p">,</span>
<span class="n">merged_reasons</span><span class="p">)</span>
<div class="viewcode-block" id="Summary.summarize"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.Summary.summarize">[docs]</a> <span class="k">def</span> <span class="nf">summarize</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">print_partition</span><span class="p">(</span><span class="n">indent</span><span class="p">,</span> <span class="n">desc</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">total</span><span class="p">):</span>
<span class="nb">print</span><span class="p">(</span><span class="s2">&quot;</span><span class="si">%s%d</span><span class="s2"> </span><span class="si">%s</span><span class="s2"> (</span><span class="si">%.1f%%</span><span class="s2">)&quot;</span> <span class="o">%</span> <span class="p">(</span><span class="s2">&quot; &quot;</span> <span class="o">*</span> <span class="n">indent</span><span class="p">,</span> <span class="n">n</span><span class="p">,</span> <span class="n">desc</span><span class="p">,</span> <span class="n">n</span> <span class="o">/</span> <span class="n">total</span> <span class="o">*</span> <span class="mi">100</span><span class="p">))</span>
<span class="nb">print</span><span class="p">()</span>
<span class="nb">print</span><span class="p">(</span><span class="s2">&quot;</span><span class="si">%d</span><span class="s2"> total test cases:&quot;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">tries</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="bp">self</span><span class="o">.</span><span class="n">tries</span><span class="p">:</span>
<span class="k">return</span>
<span class="n">print_partition</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s2">&quot;skipped&quot;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">skipped</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">tries</span><span class="p">)</span>
<span class="k">for</span> <span class="n">error</span><span class="p">,</span> <span class="n">reasons</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">error_reasons</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">print_partition</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="n">error</span><span class="p">,</span> <span class="nb">len</span><span class="p">(</span><span class="n">reasons</span><span class="p">),</span> <span class="bp">self</span><span class="o">.</span><span class="n">tries</span><span class="p">)</span>
<span class="n">reason_counts</span> <span class="o">=</span> <span class="nb">sorted</span><span class="p">(</span>
<span class="n">collections</span><span class="o">.</span><span class="n">Counter</span><span class="p">(</span><span class="n">reasons</span><span class="p">)</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">x</span><span class="p">:</span> <span class="n">x</span><span class="p">[</span><span class="mi">1</span><span class="p">],</span>
<span class="n">reverse</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="k">for</span> <span class="n">desc</span><span class="p">,</span> <span class="n">count</span> <span class="ow">in</span> <span class="n">reason_counts</span><span class="p">:</span>
<span class="n">print_partition</span><span class="p">(</span><span class="mi">2</span><span class="p">,</span> <span class="n">desc</span><span class="p">,</span> <span class="n">count</span><span class="p">,</span> <span class="nb">len</span><span class="p">(</span><span class="n">reasons</span><span class="p">))</span>
<span class="n">print_partition</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="s2">&quot;failed&quot;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">failures</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">tries</span><span class="p">)</span>
<span class="n">print_partition</span><span class="p">(</span>
<span class="mi">1</span><span class="p">,</span>
<span class="s2">&quot;passed&quot;</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">tries</span> <span class="o">-</span> <span class="bp">self</span><span class="o">.</span><span class="n">skipped</span> <span class="o">-</span>
<span class="nb">sum</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">reasons</span><span class="p">)</span>
<span class="k">for</span> <span class="n">reasons</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">error_reasons</span><span class="o">.</span><span class="n">values</span><span class="p">())</span> <span class="o">-</span> <span class="bp">self</span><span class="o">.</span><span class="n">failures</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">tries</span><span class="p">)</span>
<span class="nb">print</span><span class="p">()</span></div></div>
<div class="viewcode-block" id="parse_rst_ipython_tests"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.parse_rst_ipython_tests">[docs]</a><span class="k">def</span> <span class="nf">parse_rst_ipython_tests</span><span class="p">(</span><span class="n">rst</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">extraglobs</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">optionflags</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Extracts examples from an rst file and produce a test suite by running</span>
<span class="sd"> them through pandas to get the expected outputs.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Optional dependency.</span>
<span class="kn">import</span> <span class="nn">IPython</span>
<span class="kn">from</span> <span class="nn">traitlets.config</span> <span class="kn">import</span> <span class="n">Config</span>
<span class="k">def</span> <span class="nf">get_indent</span><span class="p">(</span><span class="n">line</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">len</span><span class="p">(</span><span class="n">line</span><span class="p">)</span> <span class="o">-</span> <span class="nb">len</span><span class="p">(</span><span class="n">line</span><span class="o">.</span><span class="n">lstrip</span><span class="p">())</span>
<span class="k">def</span> <span class="nf">is_example_line</span><span class="p">(</span><span class="n">line</span><span class="p">):</span>
<span class="n">line</span> <span class="o">=</span> <span class="n">line</span><span class="o">.</span><span class="n">strip</span><span class="p">()</span>
<span class="k">return</span> <span class="n">line</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">line</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">&#39;#&#39;</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">line</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">==</span> <span class="n">line</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> <span class="o">==</span> <span class="s1">&#39;:&#39;</span>
<span class="n">IMPORT_PANDAS</span> <span class="o">=</span> <span class="s1">&#39;import pandas as pd&#39;</span>
<span class="n">example_srcs</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">lines</span> <span class="o">=</span> <span class="nb">iter</span><span class="p">([(</span><span class="n">lineno</span><span class="p">,</span> <span class="n">line</span><span class="o">.</span><span class="n">rstrip</span><span class="p">())</span> <span class="k">for</span> <span class="n">lineno</span><span class="p">,</span>
<span class="n">line</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">rst</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">))</span> <span class="k">if</span> <span class="n">is_example_line</span><span class="p">(</span><span class="n">line</span><span class="p">)]</span> <span class="o">+</span>
<span class="p">[(</span><span class="kc">None</span><span class="p">,</span> <span class="s1">&#39;END&#39;</span><span class="p">)])</span>
<span class="c1"># https://ipython.readthedocs.io/en/stable/sphinxext.html</span>
<span class="n">lineno</span><span class="p">,</span> <span class="n">line</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">lines</span><span class="p">)</span>
<span class="k">while</span> <span class="kc">True</span><span class="p">:</span>
<span class="k">if</span> <span class="n">line</span> <span class="o">==</span> <span class="s1">&#39;END&#39;</span><span class="p">:</span>
<span class="k">break</span>
<span class="k">if</span> <span class="n">line</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s1">&#39;.. ipython::&#39;</span><span class="p">):</span>
<span class="n">lineno</span><span class="p">,</span> <span class="n">line</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">lines</span><span class="p">)</span>
<span class="n">indent</span> <span class="o">=</span> <span class="n">get_indent</span><span class="p">(</span><span class="n">line</span><span class="p">)</span>
<span class="n">example</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">example_srcs</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">lineno</span><span class="p">,</span> <span class="n">example</span><span class="p">))</span>
<span class="k">while</span> <span class="n">get_indent</span><span class="p">(</span><span class="n">line</span><span class="p">)</span> <span class="o">&gt;=</span> <span class="n">indent</span><span class="p">:</span>
<span class="k">if</span> <span class="s1">&#39;@verbatim&#39;</span> <span class="ow">in</span> <span class="n">line</span> <span class="ow">or</span> <span class="s1">&#39;:verbatim:&#39;</span> <span class="ow">in</span> <span class="n">line</span> <span class="ow">or</span> <span class="s1">&#39;@savefig&#39;</span> <span class="ow">in</span> <span class="n">line</span><span class="p">:</span>
<span class="n">example_srcs</span><span class="o">.</span><span class="n">pop</span><span class="p">()</span>
<span class="k">break</span>
<span class="n">line</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">sub</span><span class="p">(</span><span class="sa">r</span><span class="s1">&#39;In \[\d+\]: &#39;</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">,</span> <span class="n">line</span><span class="p">)</span>
<span class="n">line</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">sub</span><span class="p">(</span><span class="sa">r</span><span class="s1">&#39;\.\.\.+:&#39;</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">,</span> <span class="n">line</span><span class="p">)</span>
<span class="n">example</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">line</span><span class="p">[</span><span class="n">indent</span><span class="p">:])</span>
<span class="n">lineno</span><span class="p">,</span> <span class="n">line</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">lines</span><span class="p">)</span>
<span class="k">if</span> <span class="n">get_indent</span><span class="p">(</span><span class="n">line</span><span class="p">)</span> <span class="o">==</span> <span class="n">indent</span> <span class="ow">and</span> <span class="n">line</span><span class="p">[</span><span class="n">indent</span><span class="p">]</span> <span class="ow">not</span> <span class="ow">in</span> <span class="s1">&#39;)]}&#39;</span><span class="p">:</span>
<span class="n">example</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">example_srcs</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">lineno</span><span class="p">,</span> <span class="n">example</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">lineno</span><span class="p">,</span> <span class="n">line</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">lines</span><span class="p">)</span>
<span class="c1"># TODO(robertwb): Would it be better to try and detect/compare the actual</span>
<span class="c1"># objects in two parallel sessions than make (stringified) doctests?</span>
<span class="n">examples</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">config</span> <span class="o">=</span> <span class="n">Config</span><span class="p">()</span>
<span class="n">config</span><span class="o">.</span><span class="n">HistoryManager</span><span class="o">.</span><span class="n">hist_file</span> <span class="o">=</span> <span class="s1">&#39;:memory:&#39;</span>
<span class="n">config</span><span class="o">.</span><span class="n">InteractiveShell</span><span class="o">.</span><span class="n">autocall</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">config</span><span class="o">.</span><span class="n">InteractiveShell</span><span class="o">.</span><span class="n">autoindent</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">config</span><span class="o">.</span><span class="n">InteractiveShell</span><span class="o">.</span><span class="n">colors</span> <span class="o">=</span> <span class="s1">&#39;NoColor&#39;</span>
<span class="n">set_pandas_options</span><span class="p">()</span>
<span class="n">IP</span> <span class="o">=</span> <span class="n">IPython</span><span class="o">.</span><span class="n">InteractiveShell</span><span class="o">.</span><span class="n">instance</span><span class="p">(</span><span class="n">config</span><span class="o">=</span><span class="n">config</span><span class="p">)</span>
<span class="n">IP</span><span class="o">.</span><span class="n">run_cell</span><span class="p">(</span><span class="n">IMPORT_PANDAS</span> <span class="o">+</span> <span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">)</span>
<span class="n">IP</span><span class="o">.</span><span class="n">run_cell</span><span class="p">(</span><span class="s1">&#39;import numpy as np</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">stdout</span> <span class="o">=</span> <span class="n">sys</span><span class="o">.</span><span class="n">stdout</span>
<span class="k">for</span> <span class="n">lineno</span><span class="p">,</span> <span class="n">src</span> <span class="ow">in</span> <span class="n">example_srcs</span><span class="p">:</span>
<span class="n">sys</span><span class="o">.</span><span class="n">stdout</span> <span class="o">=</span> <span class="n">cout</span> <span class="o">=</span> <span class="n">StringIO</span><span class="p">()</span>
<span class="n">src</span> <span class="o">=</span> <span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">src</span><span class="p">)</span>
<span class="k">if</span> <span class="n">src</span> <span class="o">==</span> <span class="n">IMPORT_PANDAS</span><span class="p">:</span>
<span class="k">continue</span>
<span class="n">IP</span><span class="o">.</span><span class="n">run_cell</span><span class="p">(</span><span class="n">src</span> <span class="o">+</span> <span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">)</span>
<span class="n">output</span> <span class="o">=</span> <span class="n">cout</span><span class="o">.</span><span class="n">getvalue</span><span class="p">()</span>
<span class="k">if</span> <span class="n">output</span><span class="p">:</span>
<span class="c1"># Strip the prompt.</span>
<span class="c1"># TODO(robertwb): Figure out how to suppress this.</span>
<span class="n">output</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">sub</span><span class="p">(</span><span class="sa">r</span><span class="s1">&#39;^Out\[\d+\]:[ \t]*\n?&#39;</span><span class="p">,</span> <span class="s1">&#39;&#39;</span><span class="p">,</span> <span class="n">output</span><span class="p">)</span>
<span class="n">examples</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">doctest</span><span class="o">.</span><span class="n">Example</span><span class="p">(</span><span class="n">src</span><span class="p">,</span> <span class="n">output</span><span class="p">,</span> <span class="n">lineno</span><span class="o">=</span><span class="n">lineno</span><span class="p">))</span>
<span class="k">finally</span><span class="p">:</span>
<span class="n">sys</span><span class="o">.</span><span class="n">stdout</span> <span class="o">=</span> <span class="n">stdout</span>
<span class="k">return</span> <span class="n">doctest</span><span class="o">.</span><span class="n">DocTest</span><span class="p">(</span>
<span class="n">examples</span><span class="p">,</span> <span class="nb">dict</span><span class="p">(</span><span class="n">extraglobs</span> <span class="ow">or</span> <span class="p">{},</span> <span class="n">np</span><span class="o">=</span><span class="n">np</span><span class="p">),</span> <span class="n">name</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span></div>
<div class="viewcode-block" id="test_rst_ipython"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.test_rst_ipython">[docs]</a><span class="k">def</span> <span class="nf">test_rst_ipython</span><span class="p">(</span>
<span class="n">rst</span><span class="p">,</span>
<span class="n">name</span><span class="p">,</span>
<span class="n">report</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span>
<span class="n">wont_implement_ok</span><span class="o">=</span><span class="p">(),</span>
<span class="n">not_implemented_ok</span><span class="o">=</span><span class="p">(),</span>
<span class="n">skip</span><span class="o">=</span><span class="p">(),</span>
<span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Extracts examples from an rst file and run them through pandas to get the</span>
<span class="sd"> expected output, and then compare them against our dataframe implementation.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">run_tests</span><span class="p">(</span><span class="n">extraglobs</span><span class="p">,</span> <span class="n">optionflags</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="c1"># The patched one.</span>
<span class="n">tests</span> <span class="o">=</span> <span class="n">parse_rst_ipython_tests</span><span class="p">(</span><span class="n">rst</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">extraglobs</span><span class="p">,</span> <span class="n">optionflags</span><span class="p">)</span>
<span class="n">runner</span> <span class="o">=</span> <span class="n">doctest</span><span class="o">.</span><span class="n">DocTestRunner</span><span class="p">(</span><span class="n">optionflags</span><span class="o">=</span><span class="n">optionflags</span><span class="p">)</span>
<span class="n">set_pandas_options</span><span class="p">()</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">runner</span><span class="o">.</span><span class="n">run</span><span class="p">(</span><span class="n">tests</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">report</span><span class="p">:</span>
<span class="n">runner</span><span class="o">.</span><span class="n">summarize</span><span class="p">()</span>
<span class="k">return</span> <span class="n">result</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">_run_patched</span><span class="p">(</span>
<span class="n">run_tests</span><span class="p">,</span>
<span class="n">wont_implement_ok</span><span class="o">=</span><span class="p">{</span><span class="n">name</span><span class="p">:</span> <span class="n">wont_implement_ok</span><span class="p">},</span>
<span class="n">not_implemented_ok</span><span class="o">=</span><span class="p">{</span><span class="n">name</span><span class="p">:</span> <span class="n">not_implemented_ok</span><span class="p">},</span>
<span class="n">skip</span><span class="o">=</span><span class="p">{</span><span class="n">name</span><span class="p">:</span> <span class="n">skip</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">result</span></div>
<div class="viewcode-block" id="teststring"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.teststring">[docs]</a><span class="k">def</span> <span class="nf">teststring</span><span class="p">(</span><span class="n">text</span><span class="p">,</span> <span class="n">wont_implement_ok</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">not_implemented_ok</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="k">return</span> <span class="n">teststrings</span><span class="p">(</span>
<span class="p">{</span><span class="s1">&#39;&lt;string&gt;&#39;</span><span class="p">:</span> <span class="n">text</span><span class="p">},</span>
<span class="n">wont_implement_ok</span><span class="o">=</span><span class="p">{</span><span class="s1">&#39;&lt;string&gt;&#39;</span><span class="p">:</span> <span class="p">[</span><span class="s1">&#39;*&#39;</span><span class="p">]}</span> <span class="k">if</span> <span class="n">wont_implement_ok</span> <span class="k">else</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">not_implemented_ok</span><span class="o">=</span><span class="p">{</span><span class="s1">&#39;&lt;string&gt;&#39;</span><span class="p">:</span> <span class="p">[</span><span class="s1">&#39;*&#39;</span><span class="p">]}</span> <span class="k">if</span> <span class="n">not_implemented_ok</span> <span class="k">else</span> <span class="kc">None</span><span class="p">,</span>
<span class="o">**</span><span class="n">kwargs</span><span class="p">)</span></div>
<div class="viewcode-block" id="teststrings"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.teststrings">[docs]</a><span class="k">def</span> <span class="nf">teststrings</span><span class="p">(</span><span class="n">texts</span><span class="p">,</span> <span class="n">report</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="o">**</span><span class="n">runner_kwargs</span><span class="p">):</span>
<span class="n">optionflags</span> <span class="o">=</span> <span class="n">runner_kwargs</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="s1">&#39;optionflags&#39;</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span>
<span class="n">optionflags</span> <span class="o">|=</span> <span class="p">(</span>
<span class="n">doctest</span><span class="o">.</span><span class="n">NORMALIZE_WHITESPACE</span> <span class="o">|</span> <span class="n">doctest</span><span class="o">.</span><span class="n">IGNORE_EXCEPTION_DETAIL</span><span class="p">)</span>
<span class="n">parser</span> <span class="o">=</span> <span class="n">doctest</span><span class="o">.</span><span class="n">DocTestParser</span><span class="p">()</span>
<span class="n">runner</span> <span class="o">=</span> <span class="n">BeamDataframeDoctestRunner</span><span class="p">(</span>
<span class="n">TestEnvironment</span><span class="p">(),</span> <span class="n">optionflags</span><span class="o">=</span><span class="n">optionflags</span><span class="p">,</span> <span class="o">**</span><span class="n">runner_kwargs</span><span class="p">)</span>
<span class="n">globs</span> <span class="o">=</span> <span class="p">{</span>
<span class="s1">&#39;pd&#39;</span><span class="p">:</span> <span class="n">runner</span><span class="o">.</span><span class="n">fake_pandas_module</span><span class="p">(),</span>
<span class="s1">&#39;np&#39;</span><span class="p">:</span> <span class="n">np</span><span class="p">,</span>
<span class="s1">&#39;option_context&#39;</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">option_context</span><span class="p">,</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">for</span> <span class="n">name</span><span class="p">,</span> <span class="n">text</span> <span class="ow">in</span> <span class="n">texts</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">test</span> <span class="o">=</span> <span class="n">parser</span><span class="o">.</span><span class="n">get_doctest</span><span class="p">(</span><span class="n">text</span><span class="p">,</span> <span class="n">globs</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="n">name</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span>
<span class="n">runner</span><span class="o">.</span><span class="n">run</span><span class="p">(</span><span class="n">test</span><span class="p">)</span>
<span class="k">if</span> <span class="n">report</span><span class="p">:</span>
<span class="n">runner</span><span class="o">.</span><span class="n">summarize</span><span class="p">()</span>
<span class="k">return</span> <span class="n">runner</span><span class="o">.</span><span class="n">summary</span><span class="p">()</span><span class="o">.</span><span class="n">result</span><span class="p">()</span></div>
<div class="viewcode-block" id="set_pandas_options"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.set_pandas_options">[docs]</a><span class="k">def</span> <span class="nf">set_pandas_options</span><span class="p">():</span>
<span class="c1"># See</span>
<span class="c1"># https://github.com/pandas-dev/pandas/blob/a00202d12d399662b8045a8dd3fdac04f18e1e55/doc/source/conf.py#L319</span>
<span class="n">np</span><span class="o">.</span><span class="n">random</span><span class="o">.</span><span class="n">seed</span><span class="p">(</span><span class="mi">123456</span><span class="p">)</span>
<span class="n">np</span><span class="o">.</span><span class="n">set_printoptions</span><span class="p">(</span><span class="n">precision</span><span class="o">=</span><span class="mi">4</span><span class="p">,</span> <span class="n">suppress</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">pd</span><span class="o">.</span><span class="n">options</span><span class="o">.</span><span class="n">display</span><span class="o">.</span><span class="n">max_rows</span> <span class="o">=</span> <span class="mi">15</span></div>
<span class="k">def</span> <span class="nf">_run_patched</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">set_pandas_options</span><span class="p">()</span>
<span class="c1"># https://github.com/pandas-dev/pandas/blob/1.0.x/setup.cfg#L63</span>
<span class="n">optionflags</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">&#39;optionflags&#39;</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span>
<span class="n">optionflags</span> <span class="o">|=</span> <span class="p">(</span>
<span class="n">doctest</span><span class="o">.</span><span class="n">NORMALIZE_WHITESPACE</span> <span class="o">|</span> <span class="n">doctest</span><span class="o">.</span><span class="n">IGNORE_EXCEPTION_DETAIL</span><span class="p">)</span>
<span class="n">env</span> <span class="o">=</span> <span class="n">TestEnvironment</span><span class="p">()</span>
<span class="n">use_beam</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">&#39;use_beam&#39;</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span>
<span class="n">skip</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">&#39;skip&#39;</span><span class="p">,</span> <span class="p">{})</span>
<span class="n">wont_implement_ok</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">&#39;wont_implement_ok&#39;</span><span class="p">,</span> <span class="p">{})</span>
<span class="n">not_implemented_ok</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">&#39;not_implemented_ok&#39;</span><span class="p">,</span> <span class="p">{})</span>
<span class="n">extraglobs</span> <span class="o">=</span> <span class="nb">dict</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">&#39;extraglobs&#39;</span><span class="p">,</span> <span class="p">{}))</span>
<span class="n">extraglobs</span><span class="p">[</span><span class="s1">&#39;pd&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">env</span><span class="o">.</span><span class="n">fake_pandas_module</span><span class="p">()</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># Unfortunately the runner is not injectable.</span>
<span class="n">original_doc_test_runner</span> <span class="o">=</span> <span class="n">doctest</span><span class="o">.</span><span class="n">DocTestRunner</span>
<span class="n">doctest</span><span class="o">.</span><span class="n">DocTestRunner</span> <span class="o">=</span> <span class="k">lambda</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">BeamDataframeDoctestRunner</span><span class="p">(</span>
<span class="n">env</span><span class="p">,</span>
<span class="n">use_beam</span><span class="o">=</span><span class="n">use_beam</span><span class="p">,</span>
<span class="n">wont_implement_ok</span><span class="o">=</span><span class="n">wont_implement_ok</span><span class="p">,</span>
<span class="n">not_implemented_ok</span><span class="o">=</span><span class="n">not_implemented_ok</span><span class="p">,</span>
<span class="n">skip</span><span class="o">=</span><span class="n">skip</span><span class="p">,</span>
<span class="o">**</span><span class="n">kwargs</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">func</span><span class="p">(</span>
<span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="n">extraglobs</span><span class="o">=</span><span class="n">extraglobs</span><span class="p">,</span> <span class="n">optionflags</span><span class="o">=</span><span class="n">optionflags</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">finally</span><span class="p">:</span>
<span class="n">doctest</span><span class="o">.</span><span class="n">DocTestRunner</span> <span class="o">=</span> <span class="n">original_doc_test_runner</span>
<div class="viewcode-block" id="with_run_patched_docstring"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.with_run_patched_docstring">[docs]</a><span class="k">def</span> <span class="nf">with_run_patched_docstring</span><span class="p">(</span><span class="n">target</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">assert</span> <span class="n">target</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">wrapper</span><span class="p">(</span><span class="n">fn</span><span class="p">):</span>
<span class="n">fn</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="sa">f</span><span class="s2">&quot;&quot;&quot;Run all pandas doctests in the specified </span><span class="si">{</span><span class="n">target</span><span class="si">}</span><span class="s2">.</span>
<span class="s2"> Arguments `skip`, `wont_implement_ok`, `not_implemented_ok` are all in the</span>
<span class="s2"> format::</span>
<span class="s2"> </span><span class="se">{{</span>
<span class="s2"> &quot;module.Class.method&quot;: [&#39;*&#39;],</span>
<span class="s2"> &quot;module.Class.other_method&quot;: [</span>
<span class="s2"> &#39;instance.other_method(bad_input)&#39;,</span>
<span class="s2"> &#39;observe_result_of_bad_input()&#39;,</span>
<span class="s2"> ],</span>
<span class="s2"> </span><span class="se">}}</span>
<span class="s2"> `&#39;*&#39;` indicates all examples should be matched, otherwise the list is a list</span>
<span class="s2"> of specific input strings that should be matched.</span>
<span class="s2"> All arguments are kwargs.</span>
<span class="s2"> Args:</span>
<span class="s2"> optionflags (int): Passed through to doctests.</span>
<span class="s2"> extraglobs (Dict[str,Any]): Passed through to doctests.</span>
<span class="s2"> use_beam (bool): If true, run a Beam pipeline with partitioned input to</span>
<span class="s2"> verify the examples, else use PartitioningSession to simulate</span>
<span class="s2"> distributed execution.</span>
<span class="s2"> skip (Dict[str,str]): A set of examples to skip entirely.</span>
<span class="s2"> wont_implement_ok (Dict[str,str]): A set of examples that are allowed to</span>
<span class="s2"> raise WontImplementError.</span>
<span class="s2"> not_implemented_ok (Dict[str,str]): A set of examples that are allowed to</span>
<span class="s2"> raise NotImplementedError.</span>
<span class="s2"> Returns:</span>
<span class="s2"> ~doctest.TestResults: A doctest result describing the passed/failed tests.</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">fn</span>
<span class="k">return</span> <span class="n">wrapper</span></div>
<div class="viewcode-block" id="testfile"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.testfile">[docs]</a><span class="nd">@with_run_patched_docstring</span><span class="p">(</span><span class="n">target</span><span class="o">=</span><span class="s2">&quot;file&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">testfile</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">_run_patched</span><span class="p">(</span><span class="n">doctest</span><span class="o">.</span><span class="n">testfile</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="testmod"><a class="viewcode-back" href="../../../apache_beam.dataframe.doctests.html#apache_beam.dataframe.doctests.testmod">[docs]</a><span class="nd">@with_run_patched_docstring</span><span class="p">(</span><span class="n">target</span><span class="o">=</span><span class="s2">&quot;module&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">testmod</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">_run_patched</span><span class="p">(</span><span class="n">doctest</span><span class="o">.</span><span class="n">testmod</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>
</pre></div>
</div>
</div>
<footer>
<hr/>
<div role="contentinfo">
<p>
&copy; Copyright
</p>
</div>
Built with <a href="http://sphinx-doc.org/">Sphinx</a> using a <a href="https://github.com/rtfd/sphinx_rtd_theme">theme</a> provided by <a href="https://readthedocs.org">Read the Docs</a>.
</footer>
</div>
</div>
</section>
</div>
<script type="text/javascript">
jQuery(function () {
SphinxRtdTheme.Navigation.enable(true);
});
</script>
</body>
</html>