blob: ef9922a476873bec0774cbaeef440e1f7416072d [file] [log] [blame]
<!DOCTYPE html>
<html>
<head>
<meta charset="utf-8" />
<title>pyspark.sql.observation &#8212; PySpark 3.5.3 documentation</title>
<link href="../../../_static/styles/theme.css?digest=1999514e3f237ded88cf" rel="stylesheet">
<link href="../../../_static/styles/pydata-sphinx-theme.css?digest=1999514e3f237ded88cf" rel="stylesheet">
<link rel="stylesheet"
href="../../../_static/vendor/fontawesome/5.13.0/css/all.min.css">
<link rel="preload" as="font" type="font/woff2" crossorigin
href="../../../_static/vendor/fontawesome/5.13.0/webfonts/fa-solid-900.woff2">
<link rel="preload" as="font" type="font/woff2" crossorigin
href="../../../_static/vendor/fontawesome/5.13.0/webfonts/fa-brands-400.woff2">
<link rel="stylesheet" href="../../../_static/styles/pydata-sphinx-theme.css" type="text/css" />
<link rel="stylesheet" href="../../../_static/pygments.css" type="text/css" />
<link rel="stylesheet" type="text/css" href="../../../_static/copybutton.css" />
<link rel="stylesheet" type="text/css" href="../../../_static/css/pyspark.css" />
<link rel="preload" as="script" href="../../../_static/scripts/pydata-sphinx-theme.js?digest=1999514e3f237ded88cf">
<script id="documentation_options" data-url_root="../../../" src="../../../_static/documentation_options.js"></script>
<script src="../../../_static/jquery.js"></script>
<script src="../../../_static/underscore.js"></script>
<script src="../../../_static/doctools.js"></script>
<script src="../../../_static/language_data.js"></script>
<script src="../../../_static/clipboard.min.js"></script>
<script src="../../../_static/copybutton.js"></script>
<script crossorigin="anonymous" integrity="sha256-Ae2Vz/4ePdIu6ZyI/5ZGsYnb+m0JlOmKPjt6XZ9JJkA=" src="https://cdnjs.cloudflare.com/ajax/libs/require.js/2.3.4/require.min.js"></script>
<script async="async" src="https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.5/latest.js?config=TeX-AMS-MML_HTMLorMML"></script>
<script type="text/x-mathjax-config">MathJax.Hub.Config({"tex2jax": {"inlineMath": [["$", "$"], ["\\(", "\\)"]], "processEscapes": true, "ignoreClass": "document", "processClass": "math|output_area"}})</script>
<link rel="canonical" href="https://spark.apache.org/docs/latest/api/python/_modules/pyspark/sql/observation.html" />
<link rel="search" title="Search" href="../../../search.html" />
<meta name="viewport" content="width=device-width, initial-scale=1" />
<meta name="docsearch:language" content="None">
<!-- Google Analytics -->
</head>
<body data-spy="scroll" data-target="#bd-toc-nav" data-offset="80">
<div class="container-fluid" id="banner"></div>
<nav class="navbar navbar-light navbar-expand-lg bg-light fixed-top bd-navbar" id="navbar-main"><div class="container-xl">
<div id="navbar-start">
<a class="navbar-brand" href="../../../index.html">
<img src="../../../_static/spark-logo-reverse.png" class="logo" alt="logo">
</a>
</div>
<button class="navbar-toggler" type="button" data-toggle="collapse" data-target="#navbar-collapsible" aria-controls="navbar-collapsible" aria-expanded="false" aria-label="Toggle navigation">
<span class="navbar-toggler-icon"></span>
</button>
<div id="navbar-collapsible" class="col-lg-9 collapse navbar-collapse">
<div id="navbar-center" class="mr-auto">
<div class="navbar-center-item">
<ul id="navbar-main-elements" class="navbar-nav">
<li class="toctree-l1 nav-item">
<a class="reference internal nav-link" href="../../../index.html">
Overview
</a>
</li>
<li class="toctree-l1 nav-item">
<a class="reference internal nav-link" href="../../../getting_started/index.html">
Getting Started
</a>
</li>
<li class="toctree-l1 nav-item">
<a class="reference internal nav-link" href="../../../user_guide/index.html">
User Guides
</a>
</li>
<li class="toctree-l1 nav-item">
<a class="reference internal nav-link" href="../../../reference/index.html">
API Reference
</a>
</li>
<li class="toctree-l1 nav-item">
<a class="reference internal nav-link" href="../../../development/index.html">
Development
</a>
</li>
<li class="toctree-l1 nav-item">
<a class="reference internal nav-link" href="../../../migration_guide/index.html">
Migration Guides
</a>
</li>
</ul>
</div>
</div>
<div id="navbar-end">
<div class="navbar-end-item">
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->
<div id="version-button" class="dropdown">
<button type="button" class="btn btn-secondary btn-sm navbar-btn dropdown-toggle" id="version_switcher_button" data-toggle="dropdown">
3.5.3
<span class="caret"></span>
</button>
<div id="version_switcher" class="dropdown-menu list-group-flush py-0" aria-labelledby="version_switcher_button">
<!-- dropdown will be populated by javascript on page load -->
</div>
</div>
<script type="text/javascript">
// Function to construct the target URL from the JSON components
function buildURL(entry) {
var template = "https://spark.apache.org/docs/{version}/api/python/index.html"; // supplied by jinja
template = template.replace("{version}", entry.version);
return template;
}
// Function to check if corresponding page path exists in other version of docs
// and, if so, go there instead of the homepage of the other docs version
function checkPageExistsAndRedirect(event) {
const currentFilePath = "_modules/pyspark/sql/observation.html",
otherDocsHomepage = event.target.getAttribute("href");
let tryUrl = `${otherDocsHomepage}${currentFilePath}`;
$.ajax({
type: 'HEAD',
url: tryUrl,
// if the page exists, go there
success: function() {
location.href = tryUrl;
}
}).fail(function() {
location.href = otherDocsHomepage;
});
return false;
}
// Function to populate the version switcher
(function () {
// get JSON config
$.getJSON("https://spark.apache.org/static/versions.json", function(data, textStatus, jqXHR) {
// create the nodes first (before AJAX calls) to ensure the order is
// correct (for now, links will go to doc version homepage)
$.each(data, function(index, entry) {
// if no custom name specified (e.g., "latest"), use version string
if (!("name" in entry)) {
entry.name = entry.version;
}
// construct the appropriate URL, and add it to the dropdown
entry.url = buildURL(entry);
const node = document.createElement("a");
node.setAttribute("class", "list-group-item list-group-item-action py-1");
node.setAttribute("href", `${entry.url}`);
node.textContent = `${entry.name}`;
node.onclick = checkPageExistsAndRedirect;
$("#version_switcher").append(node);
});
});
})();
</script>
</div>
</div>
</div>
</div>
</nav>
<div class="container-xl">
<div class="row">
<!-- Only show if we have sidebars configured, else just a small margin -->
<div class="col-12 col-md-3 bd-sidebar">
<div class="sidebar-start-items"><form class="bd-search d-flex align-items-center" action="../../../search.html" method="get">
<i class="icon fas fa-search"></i>
<input type="search" class="form-control" name="q" id="search-input" placeholder="Search the docs ..." aria-label="Search the docs ..." autocomplete="off" >
</form><nav class="bd-links" id="bd-docs-nav" aria-label="Main navigation">
<div class="bd-toc-item active">
</div>
</nav>
</div>
<div class="sidebar-end-items">
</div>
</div>
<div class="d-none d-xl-block col-xl-2 bd-toc">
</div>
<main class="col-12 col-md-9 col-xl-7 py-md-5 pl-md-5 pr-md-4 bd-content" role="main">
<div>
<h1>Source code for pyspark.sql.observation</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span><span class="p">,</span> <span class="n">Dict</span><span class="p">,</span> <span class="n">Optional</span>
<span class="kn">from</span> <span class="nn">py4j.java_gateway</span> <span class="kn">import</span> <span class="n">JavaObject</span><span class="p">,</span> <span class="n">JVMView</span>
<span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">column</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.column</span> <span class="kn">import</span> <span class="n">Column</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.dataframe</span> <span class="kn">import</span> <span class="n">DataFrame</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.utils</span> <span class="kn">import</span> <span class="n">try_remote_observation</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;Observation&quot;</span><span class="p">]</span>
<div class="viewcode-block" id="Observation"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.Observation.html#pyspark.sql.Observation">[docs]</a><span class="k">class</span> <span class="nc">Observation</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Class to observe (named) metrics on a :class:`DataFrame`.</span>
<span class="sd"> Metrics are aggregation expressions, which are applied to the DataFrame while it is being</span>
<span class="sd"> processed by an action.</span>
<span class="sd"> The metrics have the following guarantees:</span>
<span class="sd"> - It will compute the defined aggregates (metrics) on all the data that is flowing through</span>
<span class="sd"> the Dataset during the action.</span>
<span class="sd"> - It will report the value of the defined aggregate columns as soon as we reach the end of</span>
<span class="sd"> the action.</span>
<span class="sd"> The metrics columns must either contain a literal (e.g. lit(42)), or should contain one or</span>
<span class="sd"> more aggregate functions (e.g. sum(a) or sum(a + b) + avg(c) - lit(1)). Expressions that</span>
<span class="sd"> contain references to the input Dataset&#39;s columns must always be wrapped in an aggregate</span>
<span class="sd"> function.</span>
<span class="sd"> An Observation instance collects the metrics while the first action is executed. Subsequent</span>
<span class="sd"> actions do not modify the metrics returned by `Observation.get`. Retrieval of the metric via</span>
<span class="sd"> `Observation.get` blocks until the first action has finished and metrics become available.</span>
<span class="sd"> .. versionadded:: 3.3.0</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This class does not support streaming datasets.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql.functions import col, count, lit, max</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql import Observation</span>
<span class="sd"> &gt;&gt;&gt; df = spark.createDataFrame([[&quot;Alice&quot;, 2], [&quot;Bob&quot;, 5]], [&quot;name&quot;, &quot;age&quot;])</span>
<span class="sd"> &gt;&gt;&gt; observation = Observation(&quot;my metrics&quot;)</span>
<span class="sd"> &gt;&gt;&gt; observed_df = df.observe(observation, count(lit(1)).alias(&quot;count&quot;), max(col(&quot;age&quot;)))</span>
<span class="sd"> &gt;&gt;&gt; observed_df.count()</span>
<span class="sd"> 2</span>
<span class="sd"> &gt;&gt;&gt; observation.get</span>
<span class="sd"> {&#39;count&#39;: 2, &#39;max(age)&#39;: 5}</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">name</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Constructs a named or unnamed Observation instance.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> name : str, optional</span>
<span class="sd"> default is a random UUID string. This is the name of the Observation and the metric.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">name</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;name should be a string&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">name</span> <span class="o">==</span> <span class="s2">&quot;&quot;</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;name should not be empty&quot;</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_name</span> <span class="o">=</span> <span class="n">name</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jvm</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">JVMView</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jo</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">JavaObject</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span>
<span class="nd">@try_remote_observation</span>
<span class="k">def</span> <span class="nf">_on</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">df</span><span class="p">:</span> <span class="n">DataFrame</span><span class="p">,</span> <span class="o">*</span><span class="n">exprs</span><span class="p">:</span> <span class="n">Column</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">DataFrame</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Attaches this observation to the given :class:`DataFrame` to observe aggregations.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> df : :class:`DataFrame`</span>
<span class="sd"> the :class:`DataFrame` to be observed</span>
<span class="sd"> exprs : list of :class:`Column`</span>
<span class="sd"> column expressions (:class:`Column`).</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> :class:`DataFrame`</span>
<span class="sd"> the observed :class:`DataFrame`.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jo</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">,</span> <span class="s2">&quot;an Observation can be used with a DataFrame only once&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jvm</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">_sc</span><span class="o">.</span><span class="n">_jvm</span>
<span class="k">assert</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jvm</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="bp">cls</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">org</span><span class="o">.</span><span class="n">apache</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">Observation</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jo</span> <span class="o">=</span> <span class="bp">cls</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_name</span><span class="p">)</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_name</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">else</span> <span class="bp">cls</span><span class="p">()</span>
<span class="n">observed_df</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jo</span><span class="o">.</span><span class="n">on</span><span class="p">(</span>
<span class="n">df</span><span class="o">.</span><span class="n">_jdf</span><span class="p">,</span> <span class="n">exprs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">_jc</span><span class="p">,</span> <span class="n">column</span><span class="o">.</span><span class="n">_to_seq</span><span class="p">(</span><span class="n">df</span><span class="o">.</span><span class="n">_sc</span><span class="p">,</span> <span class="p">[</span><span class="n">c</span><span class="o">.</span><span class="n">_jc</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">exprs</span><span class="p">[</span><span class="mi">1</span><span class="p">:]])</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">observed_df</span><span class="p">,</span> <span class="n">df</span><span class="o">.</span><span class="n">sparkSession</span><span class="p">)</span>
<span class="c1"># Note that decorated property only works with Python 3.9+ which Spark Connect requires.</span>
<span class="nd">@property</span>
<span class="nd">@try_remote_observation</span>
<span class="k">def</span> <span class="nf">get</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Any</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Get the observed metrics.</span>
<span class="sd"> Waits until the observed dataset finishes its first action. Only the result of the</span>
<span class="sd"> first action is available. Subsequent actions do not modify the result.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> dict</span>
<span class="sd"> the observed metrics</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jo</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">,</span> <span class="s2">&quot;call DataFrame.observe&quot;</span>
<span class="n">jmap</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jo</span><span class="o">.</span><span class="n">getAsJava</span><span class="p">()</span>
<span class="c1"># return a pure Python dict, not jmap which is a py4j JavaMap</span>
<span class="k">return</span> <span class="p">{</span><span class="n">k</span><span class="p">:</span> <span class="n">v</span> <span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">jmap</span><span class="o">.</span><span class="n">items</span><span class="p">()}</span></div>
<span class="k">def</span> <span class="nf">_test</span><span class="p">()</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="kn">import</span> <span class="nn">doctest</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">from</span> <span class="nn">pyspark.context</span> <span class="kn">import</span> <span class="n">SparkContext</span>
<span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">SparkSession</span>
<span class="kn">import</span> <span class="nn">pyspark.sql.observation</span>
<span class="n">globs</span> <span class="o">=</span> <span class="n">pyspark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">observation</span><span class="o">.</span><span class="vm">__dict__</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">sc</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="p">(</span><span class="s2">&quot;local[4]&quot;</span><span class="p">,</span> <span class="s2">&quot;PythonTest&quot;</span><span class="p">)</span>
<span class="n">globs</span><span class="p">[</span><span class="s2">&quot;spark&quot;</span><span class="p">]</span> <span class="o">=</span> <span class="n">SparkSession</span><span class="p">(</span><span class="n">sc</span><span class="p">)</span>
<span class="p">(</span><span class="n">failure_count</span><span class="p">,</span> <span class="n">test_count</span><span class="p">)</span> <span class="o">=</span> <span class="n">doctest</span><span class="o">.</span><span class="n">testmod</span><span class="p">(</span><span class="n">pyspark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">observation</span><span class="p">,</span> <span class="n">globs</span><span class="o">=</span><span class="n">globs</span><span class="p">)</span>
<span class="n">sc</span><span class="o">.</span><span class="n">stop</span><span class="p">()</span>
<span class="k">if</span> <span class="n">failure_count</span><span class="p">:</span>
<span class="n">sys</span><span class="o">.</span><span class="n">exit</span><span class="p">(</span><span class="o">-</span><span class="mi">1</span><span class="p">)</span>
<span class="k">if</span> <span class="vm">__name__</span> <span class="o">==</span> <span class="s2">&quot;__main__&quot;</span><span class="p">:</span>
<span class="n">_test</span><span class="p">()</span>
</pre></div>
</div>
<!-- Previous / next buttons -->
<div class='prev-next-area'>
</div>
</main>
</div>
</div>
<script src="../../../_static/scripts/pydata-sphinx-theme.js?digest=1999514e3f237ded88cf"></script>
<footer class="footer mt-5 mt-md-0">
<div class="container">
<div class="footer-item">
<p class="copyright">
&copy; Copyright .<br>
</p>
</div>
<div class="footer-item">
<p class="sphinx-version">
Created using <a href="http://sphinx-doc.org/">Sphinx</a> 3.0.4.<br>
</p>
</div>
</div>
</footer>
</body>
</html>