blob: accfea72fb6ff83f9ea5fd5a8101e7b0e123bb35 [file] [log] [blame]
<!DOCTYPE html>
<html >
<head>
<meta charset="utf-8" />
<meta name="viewport" content="width=device-width, initial-scale=1.0" />
<title>pyspark.pandas.frame &#8212; PySpark 4.0.0-preview1 documentation</title>
<script data-cfasync="false">
document.documentElement.dataset.mode = localStorage.getItem("mode") || "";
document.documentElement.dataset.theme = localStorage.getItem("theme") || "light";
</script>
<!-- Loaded before other Sphinx assets -->
<link href="../../../_static/styles/theme.css?digest=e353d410970836974a52" rel="stylesheet" />
<link href="../../../_static/styles/bootstrap.css?digest=e353d410970836974a52" rel="stylesheet" />
<link href="../../../_static/styles/pydata-sphinx-theme.css?digest=e353d410970836974a52" rel="stylesheet" />
<link href="../../../_static/vendor/fontawesome/6.1.2/css/all.min.css?digest=e353d410970836974a52" rel="stylesheet" />
<link rel="preload" as="font" type="font/woff2" crossorigin href="../../../_static/vendor/fontawesome/6.1.2/webfonts/fa-solid-900.woff2" />
<link rel="preload" as="font" type="font/woff2" crossorigin href="../../../_static/vendor/fontawesome/6.1.2/webfonts/fa-brands-400.woff2" />
<link rel="preload" as="font" type="font/woff2" crossorigin href="../../../_static/vendor/fontawesome/6.1.2/webfonts/fa-regular-400.woff2" />
<link rel="stylesheet" type="text/css" href="../../../_static/pygments.css" />
<link rel="stylesheet" type="text/css" href="../../../_static/copybutton.css" />
<link rel="stylesheet" type="text/css" href="../../../_static/css/pyspark.css" />
<!-- Pre-loaded scripts that we'll load fully later -->
<link rel="preload" as="script" href="../../../_static/scripts/bootstrap.js?digest=e353d410970836974a52" />
<link rel="preload" as="script" href="../../../_static/scripts/pydata-sphinx-theme.js?digest=e353d410970836974a52" />
<script data-url_root="../../../" id="documentation_options" 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/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>DOCUMENTATION_OPTIONS.pagename = '_modules/pyspark/pandas/frame';</script>
<link rel="canonical" href="https://spark.apache.org/docs/latest/api/python/_modules/pyspark/pandas/frame.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">
<!-- Matomo -->
<script type="text/javascript">
var _paq = window._paq = window._paq || [];
/* tracker methods like "setCustomDimension" should be called before "trackPageView" */
_paq.push(["disableCookies"]);
_paq.push(['trackPageView']);
_paq.push(['enableLinkTracking']);
(function() {
var u="https://analytics.apache.org/";
_paq.push(['setTrackerUrl', u+'matomo.php']);
_paq.push(['setSiteId', '40']);
var d=document, g=d.createElement('script'), s=d.getElementsByTagName('script')[0];
g.async=true; g.src=u+'matomo.js'; s.parentNode.insertBefore(g,s);
})();
</script>
<!-- End Matomo Code -->
</head>
<body data-bs-spy="scroll" data-bs-target=".bd-toc-nav" data-offset="180" data-bs-root-margin="0px 0px -60%" data-default-mode="">
<a class="skip-link" href="#main-content">Skip to main content</a>
<input type="checkbox"
class="sidebar-toggle"
name="__primary"
id="__primary"/>
<label class="overlay overlay-primary" for="__primary"></label>
<input type="checkbox"
class="sidebar-toggle"
name="__secondary"
id="__secondary"/>
<label class="overlay overlay-secondary" for="__secondary"></label>
<div class="search-button__wrapper">
<div class="search-button__overlay"></div>
<div class="search-button__search-container">
<form class="bd-search d-flex align-items-center"
action="../../../search.html"
method="get">
<i class="fa-solid fa-magnifying-glass"></i>
<input type="search"
class="form-control"
name="q"
id="search-input"
placeholder="Search the docs ..."
aria-label="Search the docs ..."
autocomplete="off"
autocorrect="off"
autocapitalize="off"
spellcheck="false"/>
<span class="search-button__kbd-shortcut"><kbd class="kbd-shortcut__modifier">Ctrl</kbd>+<kbd>K</kbd></span>
</form></div>
</div>
<nav class="bd-header navbar navbar-expand-lg bd-navbar">
<div class="bd-header__inner bd-page-width">
<label class="sidebar-toggle primary-toggle" for="__primary">
<span class="fa-solid fa-bars"></span>
</label>
<div class="navbar-header-items__start">
<div class="navbar-item">
<a class="navbar-brand logo" href="../../../index.html">
<img src="../../../_static/spark-logo-light.png" class="logo__image only-light" alt="Logo image"/>
<script>document.write(`<img src="../../../_static/spark-logo-dark.png" class="logo__image only-dark" alt="Logo image"/>`);</script>
</a></div>
</div>
<div class="col-lg-9 navbar-header-items">
<div class="me-auto navbar-header-items__center">
<div class="navbar-item"><nav class="navbar-nav">
<p class="sidebar-header-items__title"
role="heading"
aria-level="1"
aria-label="Site Navigation">
Site Navigation
</p>
<ul class="bd-navbar-elements navbar-nav">
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../index.html">
Overview
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../getting_started/index.html">
Getting Started
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../user_guide/index.html">
User Guides
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../reference/index.html">
API Reference
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../development/index.html">
Development
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../migration_guide/index.html">
Migration Guides
</a>
</li>
</ul>
</nav></div>
</div>
<div class="navbar-header-items__end">
<div class="navbar-item navbar-persistent--container">
<script>
document.write(`
<button class="btn btn-sm navbar-btn search-button search-button__button" title="Search" aria-label="Search" data-bs-placement="bottom" data-bs-toggle="tooltip">
<i class="fa-solid fa-magnifying-glass"></i>
</button>
`);
</script>
</div>
<div class="navbar-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">
4.0.0-preview1
<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/pandas/frame.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 class="navbar-item">
<script>
document.write(`
<button class="theme-switch-button btn btn-sm btn-outline-primary navbar-btn rounded-circle" title="light/dark" aria-label="light/dark" data-bs-placement="bottom" data-bs-toggle="tooltip">
<span class="theme-switch" data-mode="light"><i class="fa-solid fa-sun"></i></span>
<span class="theme-switch" data-mode="dark"><i class="fa-solid fa-moon"></i></span>
<span class="theme-switch" data-mode="auto"><i class="fa-solid fa-circle-half-stroke"></i></span>
</button>
`);
</script></div>
<div class="navbar-item"><ul class="navbar-icon-links navbar-nav"
aria-label="Icon Links">
<li class="nav-item">
<a href="https://github.com/apache/spark" title="GitHub" class="nav-link" rel="noopener" target="_blank" data-bs-toggle="tooltip" data-bs-placement="bottom"><span><i class="fa-brands fa-github"></i></span>
<label class="sr-only">GitHub</label></a>
</li>
<li class="nav-item">
<a href="https://pypi.org/project/pyspark" title="PyPI" class="nav-link" rel="noopener" target="_blank" data-bs-toggle="tooltip" data-bs-placement="bottom"><span><i class="fa-solid fa-box"></i></span>
<label class="sr-only">PyPI</label></a>
</li>
</ul></div>
</div>
</div>
<div class="navbar-persistent--mobile">
<script>
document.write(`
<button class="btn btn-sm navbar-btn search-button search-button__button" title="Search" aria-label="Search" data-bs-placement="bottom" data-bs-toggle="tooltip">
<i class="fa-solid fa-magnifying-glass"></i>
</button>
`);
</script>
</div>
</div>
</nav>
<div class="bd-container">
<div class="bd-container__inner bd-page-width">
<div class="bd-sidebar-primary bd-sidebar hide-on-wide">
<div class="sidebar-header-items sidebar-primary__section">
<div class="sidebar-header-items__center">
<div class="navbar-item"><nav class="navbar-nav">
<p class="sidebar-header-items__title"
role="heading"
aria-level="1"
aria-label="Site Navigation">
Site Navigation
</p>
<ul class="bd-navbar-elements navbar-nav">
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../index.html">
Overview
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../getting_started/index.html">
Getting Started
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../user_guide/index.html">
User Guides
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../reference/index.html">
API Reference
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../development/index.html">
Development
</a>
</li>
<li class="nav-item">
<a class="nav-link nav-internal" href="../../../migration_guide/index.html">
Migration Guides
</a>
</li>
</ul>
</nav></div>
</div>
<div class="sidebar-header-items__end">
<div class="navbar-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">
4.0.0-preview1
<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/pandas/frame.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 class="navbar-item">
<script>
document.write(`
<button class="theme-switch-button btn btn-sm btn-outline-primary navbar-btn rounded-circle" title="light/dark" aria-label="light/dark" data-bs-placement="bottom" data-bs-toggle="tooltip">
<span class="theme-switch" data-mode="light"><i class="fa-solid fa-sun"></i></span>
<span class="theme-switch" data-mode="dark"><i class="fa-solid fa-moon"></i></span>
<span class="theme-switch" data-mode="auto"><i class="fa-solid fa-circle-half-stroke"></i></span>
</button>
`);
</script></div>
<div class="navbar-item"><ul class="navbar-icon-links navbar-nav"
aria-label="Icon Links">
<li class="nav-item">
<a href="https://github.com/apache/spark" title="GitHub" class="nav-link" rel="noopener" target="_blank" data-bs-toggle="tooltip" data-bs-placement="bottom"><span><i class="fa-brands fa-github"></i></span>
<label class="sr-only">GitHub</label></a>
</li>
<li class="nav-item">
<a href="https://pypi.org/project/pyspark" title="PyPI" class="nav-link" rel="noopener" target="_blank" data-bs-toggle="tooltip" data-bs-placement="bottom"><span><i class="fa-solid fa-box"></i></span>
<label class="sr-only">PyPI</label></a>
</li>
</ul></div>
</div>
</div>
<div class="sidebar-primary-items__end sidebar-primary__section">
</div>
<div id="rtd-footer-container"></div>
</div>
<main id="main-content" class="bd-main">
<div class="bd-content">
<div class="bd-article-container">
<div class="bd-header-article">
<div class="header-article-items header-article__inner">
<div class="header-article-items__start">
<div class="header-article-item">
<nav aria-label="Breadcrumbs">
<ul class="bd-breadcrumbs" role="navigation" aria-label="Breadcrumb">
<li class="breadcrumb-item breadcrumb-home">
<a href="../../../index.html" class="nav-link" aria-label="Home">
<i class="fa-solid fa-home"></i>
</a>
</li>
<li class="breadcrumb-item"><a href="../../index.html" class="nav-link">Module code</a></li>
<li class="breadcrumb-item active" aria-current="page">pyspark.pandas.frame</li>
</ul>
</nav>
</div>
</div>
</div>
</div>
<div id="searchbox"></div>
<article class="bd-article" role="main">
<h1>Source code for pyspark.pandas.frame</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="sd">&quot;&quot;&quot;</span>
<span class="sd">A wrapper class for Spark DataFrame to behave like pandas DataFrame.</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">collections</span> <span class="kn">import</span> <span class="n">defaultdict</span><span class="p">,</span> <span class="n">namedtuple</span>
<span class="kn">from</span> <span class="nn">collections.abc</span> <span class="kn">import</span> <span class="n">Mapping</span>
<span class="kn">import</span> <span class="nn">re</span>
<span class="kn">import</span> <span class="nn">warnings</span>
<span class="kn">import</span> <span class="nn">inspect</span>
<span class="kn">import</span> <span class="nn">json</span>
<span class="kn">import</span> <span class="nn">types</span>
<span class="kn">from</span> <span class="nn">functools</span> <span class="kn">import</span> <span class="n">partial</span><span class="p">,</span> <span class="n">reduce</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">from</span> <span class="nn">itertools</span> <span class="kn">import</span> <span class="n">zip_longest</span><span class="p">,</span> <span class="n">chain</span>
<span class="kn">from</span> <span class="nn">types</span> <span class="kn">import</span> <span class="n">TracebackType</span>
<span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="p">(</span>
<span class="n">Any</span><span class="p">,</span>
<span class="n">Callable</span><span class="p">,</span>
<span class="n">Dict</span><span class="p">,</span>
<span class="n">Generic</span><span class="p">,</span>
<span class="n">IO</span><span class="p">,</span>
<span class="n">Iterable</span><span class="p">,</span>
<span class="n">Iterator</span><span class="p">,</span>
<span class="n">List</span><span class="p">,</span>
<span class="n">Optional</span><span class="p">,</span>
<span class="n">Sequence</span><span class="p">,</span>
<span class="n">Tuple</span><span class="p">,</span>
<span class="n">Type</span><span class="p">,</span>
<span class="n">Union</span><span class="p">,</span>
<span class="n">cast</span><span class="p">,</span>
<span class="n">no_type_check</span><span class="p">,</span>
<span class="n">TYPE_CHECKING</span><span class="p">,</span>
<span class="p">)</span>
<span class="kn">import</span> <span class="nn">datetime</span>
<span class="kn">import</span> <span class="nn">numpy</span> <span class="k">as</span> <span class="nn">np</span>
<span class="kn">import</span> <span class="nn">pandas</span> <span class="k">as</span> <span class="nn">pd</span>
<span class="kn">from</span> <span class="nn">pandas.api.types</span> <span class="kn">import</span> <span class="p">(</span> <span class="c1"># type: ignore[attr-defined]</span>
<span class="n">is_bool_dtype</span><span class="p">,</span>
<span class="n">is_list_like</span><span class="p">,</span>
<span class="n">is_dict_like</span><span class="p">,</span>
<span class="n">is_scalar</span><span class="p">,</span>
<span class="p">)</span>
<span class="kn">from</span> <span class="nn">pandas.tseries.frequencies</span> <span class="kn">import</span> <span class="n">DateOffset</span><span class="p">,</span> <span class="n">to_offset</span>
<span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">pandas.io.formats.style</span> <span class="kn">import</span> <span class="n">Styler</span>
<span class="kn">from</span> <span class="nn">pandas.core.dtypes.common</span> <span class="kn">import</span> <span class="n">infer_dtype_from_object</span>
<span class="kn">from</span> <span class="nn">pandas.core.accessor</span> <span class="kn">import</span> <span class="n">CachedAccessor</span>
<span class="kn">from</span> <span class="nn">pandas.core.dtypes.inference</span> <span class="kn">import</span> <span class="n">is_sequence</span>
<span class="kn">from</span> <span class="nn">pyspark.errors</span> <span class="kn">import</span> <span class="n">PySparkValueError</span>
<span class="kn">from</span> <span class="nn">pyspark</span> <span class="kn">import</span> <span class="n">StorageLevel</span>
<span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">Column</span> <span class="k">as</span> <span class="n">PySparkColumn</span><span class="p">,</span> <span class="n">DataFrame</span> <span class="k">as</span> <span class="n">PySparkDataFrame</span><span class="p">,</span> <span class="n">functions</span> <span class="k">as</span> <span class="n">F</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.functions</span> <span class="kn">import</span> <span class="n">pandas_udf</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.types</span> <span class="kn">import</span> <span class="p">(</span>
<span class="n">ArrayType</span><span class="p">,</span>
<span class="n">BooleanType</span><span class="p">,</span>
<span class="n">DataType</span><span class="p">,</span>
<span class="n">DoubleType</span><span class="p">,</span>
<span class="n">NumericType</span><span class="p">,</span>
<span class="n">Row</span><span class="p">,</span>
<span class="n">StringType</span><span class="p">,</span>
<span class="n">StructField</span><span class="p">,</span>
<span class="n">StructType</span><span class="p">,</span>
<span class="n">DecimalType</span><span class="p">,</span>
<span class="n">TimestampType</span><span class="p">,</span>
<span class="n">TimestampNTZType</span><span class="p">,</span>
<span class="n">NullType</span><span class="p">,</span>
<span class="p">)</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.window</span> <span class="kn">import</span> <span class="n">Window</span>
<span class="kn">from</span> <span class="nn">pyspark</span> <span class="kn">import</span> <span class="n">pandas</span> <span class="k">as</span> <span class="n">ps</span> <span class="c1"># For running doctests and reference resolution in PyCharm.</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas._typing</span> <span class="kn">import</span> <span class="p">(</span>
<span class="n">Axis</span><span class="p">,</span>
<span class="n">DataFrameOrSeries</span><span class="p">,</span>
<span class="n">Dtype</span><span class="p">,</span>
<span class="n">Label</span><span class="p">,</span>
<span class="n">Name</span><span class="p">,</span>
<span class="n">Scalar</span><span class="p">,</span>
<span class="n">T</span><span class="p">,</span>
<span class="p">)</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.accessors</span> <span class="kn">import</span> <span class="n">PandasOnSparkFrameMethods</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.config</span> <span class="kn">import</span> <span class="n">option_context</span><span class="p">,</span> <span class="n">get_option</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.correlation</span> <span class="kn">import</span> <span class="p">(</span>
<span class="n">compute</span><span class="p">,</span>
<span class="n">CORRELATION_VALUE_1_COLUMN</span><span class="p">,</span>
<span class="n">CORRELATION_VALUE_2_COLUMN</span><span class="p">,</span>
<span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">,</span>
<span class="n">CORRELATION_COUNT_OUTPUT_COLUMN</span><span class="p">,</span>
<span class="p">)</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.spark</span> <span class="kn">import</span> <span class="n">functions</span> <span class="k">as</span> <span class="n">SF</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.spark.accessors</span> <span class="kn">import</span> <span class="n">SparkFrameMethods</span><span class="p">,</span> <span class="n">CachedSparkFrameMethods</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.utils</span> <span class="kn">import</span> <span class="p">(</span>
<span class="n">align_diff_frames</span><span class="p">,</span>
<span class="n">column_labels_level</span><span class="p">,</span>
<span class="n">combine_frames</span><span class="p">,</span>
<span class="n">default_session</span><span class="p">,</span>
<span class="n">is_name_like_tuple</span><span class="p">,</span>
<span class="n">is_name_like_value</span><span class="p">,</span>
<span class="n">is_testing</span><span class="p">,</span>
<span class="n">name_like_string</span><span class="p">,</span>
<span class="n">same_anchor</span><span class="p">,</span>
<span class="n">scol_for</span><span class="p">,</span>
<span class="n">validate_arguments_and_invoke_function</span><span class="p">,</span>
<span class="n">validate_axis</span><span class="p">,</span>
<span class="n">validate_bool_kwarg</span><span class="p">,</span>
<span class="n">validate_how</span><span class="p">,</span>
<span class="n">validate_mode</span><span class="p">,</span>
<span class="n">verify_temp_column_name</span><span class="p">,</span>
<span class="n">log_advice</span><span class="p">,</span>
<span class="p">)</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.generic</span> <span class="kn">import</span> <span class="n">Frame</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.internal</span> <span class="kn">import</span> <span class="p">(</span>
<span class="n">InternalField</span><span class="p">,</span>
<span class="n">InternalFrame</span><span class="p">,</span>
<span class="n">HIDDEN_COLUMNS</span><span class="p">,</span>
<span class="n">NATURAL_ORDER_COLUMN_NAME</span><span class="p">,</span>
<span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">,</span>
<span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">,</span>
<span class="n">SPARK_DEFAULT_SERIES_NAME</span><span class="p">,</span>
<span class="n">SPARK_INDEX_NAME_PATTERN</span><span class="p">,</span>
<span class="p">)</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.missing.frame</span> <span class="kn">import</span> <span class="n">MissingPandasLikeDataFrame</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.typedef.typehints</span> <span class="kn">import</span> <span class="p">(</span>
<span class="n">as_spark_type</span><span class="p">,</span>
<span class="n">infer_return_type</span><span class="p">,</span>
<span class="n">pandas_on_spark_type</span><span class="p">,</span>
<span class="n">spark_type_to_pandas_dtype</span><span class="p">,</span>
<span class="n">DataFrameType</span><span class="p">,</span>
<span class="n">SeriesType</span><span class="p">,</span>
<span class="n">ScalarType</span><span class="p">,</span>
<span class="n">create_tuple_for_frame_type</span><span class="p">,</span>
<span class="p">)</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.plot</span> <span class="kn">import</span> <span class="n">PandasOnSparkPlotAccessor</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.utils</span> <span class="kn">import</span> <span class="n">get_column_class</span><span class="p">,</span> <span class="n">get_dataframe_class</span>
<span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">pyspark.sql._typing</span> <span class="kn">import</span> <span class="n">OptionalPrimitiveType</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.groupby</span> <span class="kn">import</span> <span class="n">DataFrameGroupBy</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.resample</span> <span class="kn">import</span> <span class="n">DataFrameResampler</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.indexes</span> <span class="kn">import</span> <span class="n">Index</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="c1"># These regular expression patterns are compiled and defined here to avoid compiling the same</span>
<span class="c1"># pattern every time it is used in _repr_ and _repr_html_ in DataFrame.</span>
<span class="c1"># Two patterns basically seek the footer string from Pandas&#39;</span>
<span class="n">REPR_PATTERN</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">compile</span><span class="p">(</span><span class="sa">r</span><span class="s2">&quot;\n\n\[(?P&lt;rows&gt;[0-9]+) rows x (?P&lt;columns&gt;[0-9]+) columns\]$&quot;</span><span class="p">)</span>
<span class="n">REPR_HTML_PATTERN</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">compile</span><span class="p">(</span>
<span class="sa">r</span><span class="s2">&quot;\n\&lt;p\&gt;(?P&lt;rows&gt;[0-9]+) rows × (?P&lt;columns&gt;[0-9]+) columns\&lt;\/p\&gt;\n\&lt;\/div\&gt;$&quot;</span>
<span class="p">)</span>
<span class="n">_flex_doc_FRAME</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2">Get </span><span class="si">{desc}</span><span class="s2"> of dataframe and other, element-wise (binary operator `</span><span class="si">{op_name}</span><span class="s2">`).</span>
<span class="s2">Equivalent to ``</span><span class="si">{equiv}</span><span class="s2">``. With the reverse version, `</span><span class="si">{reverse}</span><span class="s2">`.</span>
<span class="s2">Among flexible wrappers (`add`, `sub`, `mul`, `div`) to</span>
<span class="s2">arithmetic operators: `+`, `-`, `*`, `/`, `//`.</span>
<span class="s2">Parameters</span>
<span class="s2">----------</span>
<span class="s2">other : scalar</span>
<span class="s2"> Any single data</span>
<span class="s2">Returns</span>
<span class="s2">-------</span>
<span class="s2">DataFrame</span>
<span class="s2"> Result of the arithmetic operation.</span>
<span class="s2">Examples</span>
<span class="s2">--------</span>
<span class="s2">&gt;&gt;&gt; df = ps.DataFrame({{&#39;angles&#39;: [0, 3, 4],</span>
<span class="s2">... &#39;degrees&#39;: [360, 180, 360]}},</span>
<span class="s2">... index=[&#39;circle&#39;, &#39;triangle&#39;, &#39;rectangle&#39;],</span>
<span class="s2">... columns=[&#39;angles&#39;, &#39;degrees&#39;])</span>
<span class="s2">&gt;&gt;&gt; df</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0 360</span>
<span class="s2">triangle 3 180</span>
<span class="s2">rectangle 4 360</span>
<span class="s2">Add a scalar with operator version which returns the same</span>
<span class="s2">results. Also, the reverse version.</span>
<span class="s2">&gt;&gt;&gt; df + 1</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 1 361</span>
<span class="s2">triangle 4 181</span>
<span class="s2">rectangle 5 361</span>
<span class="s2">&gt;&gt;&gt; df.add(1)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 1 361</span>
<span class="s2">triangle 4 181</span>
<span class="s2">rectangle 5 361</span>
<span class="s2">&gt;&gt;&gt; df.add(df)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0 720</span>
<span class="s2">triangle 6 360</span>
<span class="s2">rectangle 8 720</span>
<span class="s2">&gt;&gt;&gt; df + df + df</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0 1080</span>
<span class="s2">triangle 9 540</span>
<span class="s2">rectangle 12 1080</span>
<span class="s2">&gt;&gt;&gt; df.radd(1)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 1 361</span>
<span class="s2">triangle 4 181</span>
<span class="s2">rectangle 5 361</span>
<span class="s2">Divide and true divide by constant with reverse version.</span>
<span class="s2">&gt;&gt;&gt; df / 10</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0.0 36.0</span>
<span class="s2">triangle 0.3 18.0</span>
<span class="s2">rectangle 0.4 36.0</span>
<span class="s2">&gt;&gt;&gt; df.div(10)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0.0 36.0</span>
<span class="s2">triangle 0.3 18.0</span>
<span class="s2">rectangle 0.4 36.0</span>
<span class="s2">&gt;&gt;&gt; df.rdiv(10)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle inf 0.027778</span>
<span class="s2">triangle 3.333333 0.055556</span>
<span class="s2">rectangle 2.500000 0.027778</span>
<span class="s2">&gt;&gt;&gt; df.truediv(10)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0.0 36.0</span>
<span class="s2">triangle 0.3 18.0</span>
<span class="s2">rectangle 0.4 36.0</span>
<span class="s2">&gt;&gt;&gt; df.rtruediv(10)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle inf 0.027778</span>
<span class="s2">triangle 3.333333 0.055556</span>
<span class="s2">rectangle 2.500000 0.027778</span>
<span class="s2">Subtract by constant with reverse version.</span>
<span class="s2">&gt;&gt;&gt; df - 1</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle -1 359</span>
<span class="s2">triangle 2 179</span>
<span class="s2">rectangle 3 359</span>
<span class="s2">&gt;&gt;&gt; df.sub(1)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle -1 359</span>
<span class="s2">triangle 2 179</span>
<span class="s2">rectangle 3 359</span>
<span class="s2">&gt;&gt;&gt; df.rsub(1)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 1 -359</span>
<span class="s2">triangle -2 -179</span>
<span class="s2">rectangle -3 -359</span>
<span class="s2">Multiply by constant with the reverse version.</span>
<span class="s2">&gt;&gt;&gt; df * 1</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0 360</span>
<span class="s2">triangle 3 180</span>
<span class="s2">rectangle 4 360</span>
<span class="s2">&gt;&gt;&gt; df.mul(1)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0 360</span>
<span class="s2">triangle 3 180</span>
<span class="s2">rectangle 4 360</span>
<span class="s2">&gt;&gt;&gt; df.rmul(1)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0 360</span>
<span class="s2">triangle 3 180</span>
<span class="s2">rectangle 4 360</span>
<span class="s2">Floor Divide by constant with reverse version.</span>
<span class="s2">&gt;&gt;&gt; df // 10</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0.0 36.0</span>
<span class="s2">triangle 0.0 18.0</span>
<span class="s2">rectangle 0.0 36.0</span>
<span class="s2">&gt;&gt;&gt; df.floordiv(10)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0.0 36.0</span>
<span class="s2">triangle 0.0 18.0</span>
<span class="s2">rectangle 0.0 36.0</span>
<span class="s2">&gt;&gt;&gt; df.rfloordiv(10) # doctest: +SKIP</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle inf 0.0</span>
<span class="s2">triangle 3.0 0.0</span>
<span class="s2">rectangle 2.0 0.0</span>
<span class="s2">Mod by constant with reverse version.</span>
<span class="s2">&gt;&gt;&gt; df % 2</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0 0</span>
<span class="s2">triangle 1 0</span>
<span class="s2">rectangle 0 0</span>
<span class="s2">&gt;&gt;&gt; df.mod(2)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0 0</span>
<span class="s2">triangle 1 0</span>
<span class="s2">rectangle 0 0</span>
<span class="s2">&gt;&gt;&gt; df.rmod(2)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle NaN 2</span>
<span class="s2">triangle 2.0 2</span>
<span class="s2">rectangle 2.0 2</span>
<span class="s2">Power by constant with reverse version.</span>
<span class="s2">&gt;&gt;&gt; df ** 2</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0.0 129600.0</span>
<span class="s2">triangle 9.0 32400.0</span>
<span class="s2">rectangle 16.0 129600.0</span>
<span class="s2">&gt;&gt;&gt; df.pow(2)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 0.0 129600.0</span>
<span class="s2">triangle 9.0 32400.0</span>
<span class="s2">rectangle 16.0 129600.0</span>
<span class="s2">&gt;&gt;&gt; df.rpow(2)</span>
<span class="s2"> angles degrees</span>
<span class="s2">circle 1.0 2.348543e+108</span>
<span class="s2">triangle 8.0 1.532496e+54</span>
<span class="s2">rectangle 16.0 2.348543e+108</span>
<span class="s2">&quot;&quot;&quot;</span>
<div class="viewcode-block" id="DataFrame"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.html#pyspark.pandas.DataFrame">[docs]</a><span class="k">class</span> <span class="nc">DataFrame</span><span class="p">(</span><span class="n">Frame</span><span class="p">,</span> <span class="n">Generic</span><span class="p">[</span><span class="n">T</span><span class="p">]):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> pandas-on-Spark DataFrame that corresponds to pandas DataFrame logically. This holds Spark</span>
<span class="sd"> DataFrame internally.</span>
<span class="sd"> :ivar _internal: an internal immutable Frame to manage metadata.</span>
<span class="sd"> :type _internal: InternalFrame</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> data : numpy ndarray (structured or homogeneous), dict, pandas DataFrame,</span>
<span class="sd"> Spark DataFrame, pandas-on-Spark DataFrame or pandas-on-Spark Series.</span>
<span class="sd"> Dict can contain Series, arrays, constants, or list-like objects</span>
<span class="sd"> index : Index or array-like</span>
<span class="sd"> Index to use for the resulting frame. Will default to RangeIndex if</span>
<span class="sd"> no indexing information part of input data and no index provided</span>
<span class="sd"> columns : Index or array-like</span>
<span class="sd"> Column labels to use for the resulting frame. Will default to</span>
<span class="sd"> RangeIndex (0, 1, 2, ..., n) if no column labels are provided</span>
<span class="sd"> dtype : dtype, default None</span>
<span class="sd"> Data type to force. Only a single dtype is allowed. If None, infer</span>
<span class="sd"> copy : boolean, default False</span>
<span class="sd"> Copy data from inputs. Only affects DataFrame / 2d ndarray input</span>
<span class="sd"> .. versionchanged:: 3.4.0</span>
<span class="sd"> Since 3.4.0, it deals with `data` and `index` in this approach:</span>
<span class="sd"> 1, when `data` is a distributed dataset (Internal DataFrame/Spark DataFrame/</span>
<span class="sd"> pandas-on-Spark DataFrame/pandas-on-Spark Series), it will first parallelize</span>
<span class="sd"> the `index` if necessary, and then try to combine the `data` and `index`;</span>
<span class="sd"> Note that if `data` and `index` doesn&#39;t have the same anchor, then</span>
<span class="sd"> `compute.ops_on_diff_frames` should be turned on;</span>
<span class="sd"> 2, when `data` is a local dataset (Pandas DataFrame/numpy ndarray/list/etc),</span>
<span class="sd"> it will first collect the `index` to driver if necessary, and then apply</span>
<span class="sd"> the `pandas.DataFrame(...)` creation internally;</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> Constructing DataFrame from a dictionary.</span>
<span class="sd"> &gt;&gt;&gt; d = {&#39;col1&#39;: [1, 2], &#39;col2&#39;: [3, 4]}</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(data=d, columns=[&#39;col1&#39;, &#39;col2&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> col1 col2</span>
<span class="sd"> 0 1 3</span>
<span class="sd"> 1 2 4</span>
<span class="sd"> Constructing DataFrame from pandas DataFrame</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(pd.DataFrame(data=d, columns=[&#39;col1&#39;, &#39;col2&#39;]))</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> col1 col2</span>
<span class="sd"> 0 1 3</span>
<span class="sd"> 1 2 4</span>
<span class="sd"> Notice that the inferred dtype is int64.</span>
<span class="sd"> &gt;&gt;&gt; df.dtypes</span>
<span class="sd"> col1 int64</span>
<span class="sd"> col2 int64</span>
<span class="sd"> dtype: object</span>
<span class="sd"> To enforce a single dtype:</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(data=d, dtype=np.int8)</span>
<span class="sd"> &gt;&gt;&gt; df.dtypes</span>
<span class="sd"> col1 int8</span>
<span class="sd"> col2 int8</span>
<span class="sd"> dtype: object</span>
<span class="sd"> Constructing DataFrame from numpy ndarray:</span>
<span class="sd"> &gt;&gt;&gt; import numpy as np</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),</span>
<span class="sd"> ... columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> a b c d e</span>
<span class="sd"> 0 1 2 3 4 5</span>
<span class="sd"> 1 6 7 8 9 0</span>
<span class="sd"> Constructing DataFrame from numpy ndarray with Pandas index:</span>
<span class="sd"> &gt;&gt;&gt; import numpy as np</span>
<span class="sd"> &gt;&gt;&gt; import pandas as pd</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),</span>
<span class="sd"> ... index=pd.Index([1, 4]), columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> a b c d e</span>
<span class="sd"> 1 1 2 3 4 5</span>
<span class="sd"> 4 6 7 8 9 0</span>
<span class="sd"> Constructing DataFrame from numpy ndarray with pandas-on-Spark index:</span>
<span class="sd"> &gt;&gt;&gt; import numpy as np</span>
<span class="sd"> &gt;&gt;&gt; import pandas as pd</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),</span>
<span class="sd"> ... index=ps.Index([1, 4]), columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> a b c d e</span>
<span class="sd"> 1 1 2 3 4 5</span>
<span class="sd"> 4 6 7 8 9 0</span>
<span class="sd"> Constructing DataFrame from Pandas DataFrame with Pandas index:</span>
<span class="sd"> &gt;&gt;&gt; import numpy as np</span>
<span class="sd"> &gt;&gt;&gt; import pandas as pd</span>
<span class="sd"> &gt;&gt;&gt; pdf = pd.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),</span>
<span class="sd"> ... columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame(data=pdf, index=pd.Index([1, 4]))</span>
<span class="sd"> a b c d e</span>
<span class="sd"> 1 6.0 7.0 8.0 9.0 0.0</span>
<span class="sd"> 4 NaN NaN NaN NaN NaN</span>
<span class="sd"> Constructing DataFrame from Pandas DataFrame with pandas-on-Spark index:</span>
<span class="sd"> &gt;&gt;&gt; import numpy as np</span>
<span class="sd"> &gt;&gt;&gt; import pandas as pd</span>
<span class="sd"> &gt;&gt;&gt; pdf = pd.DataFrame(data=np.array([[1, 2, 3, 4, 5], [6, 7, 8, 9, 0]]),</span>
<span class="sd"> ... columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame(data=pdf, index=ps.Index([1, 4]))</span>
<span class="sd"> a b c d e</span>
<span class="sd"> 1 6.0 7.0 8.0 9.0 0.0</span>
<span class="sd"> 4 NaN NaN NaN NaN NaN</span>
<span class="sd"> Constructing DataFrame from Spark DataFrame with Pandas index:</span>
<span class="sd"> &gt;&gt;&gt; import pandas as pd</span>
<span class="sd"> &gt;&gt;&gt; sdf = spark.createDataFrame([(&quot;Data&quot;, 1), (&quot;Bricks&quot;, 2)], [&quot;x&quot;, &quot;y&quot;])</span>
<span class="sd"> &gt;&gt;&gt; with ps.option_context(&quot;compute.ops_on_diff_frames&quot;, False):</span>
<span class="sd"> ... ps.DataFrame(data=sdf, index=pd.Index([0, 1, 2]))</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: Cannot combine the series or dataframe...&#39;compute.ops_on_diff_frames&#39; option.</span>
<span class="sd"> Enable &#39;compute.ops_on_diff_frames&#39; to combine SparkDataFrame and Pandas index</span>
<span class="sd"> &gt;&gt;&gt; with ps.option_context(&quot;compute.ops_on_diff_frames&quot;, True):</span>
<span class="sd"> ... ps.DataFrame(data=sdf, index=pd.Index([0, 1, 2]))</span>
<span class="sd"> x y</span>
<span class="sd"> 0 Data 1.0</span>
<span class="sd"> 1 Bricks 2.0</span>
<span class="sd"> 2 None NaN</span>
<span class="sd"> Constructing DataFrame from Spark DataFrame with pandas-on-Spark index:</span>
<span class="sd"> &gt;&gt;&gt; import pandas as pd</span>
<span class="sd"> &gt;&gt;&gt; sdf = spark.createDataFrame([(&quot;Data&quot;, 1), (&quot;Bricks&quot;, 2)], [&quot;x&quot;, &quot;y&quot;])</span>
<span class="sd"> &gt;&gt;&gt; with ps.option_context(&quot;compute.ops_on_diff_frames&quot;, False):</span>
<span class="sd"> ... ps.DataFrame(data=sdf, index=ps.Index([0, 1, 2]))</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: Cannot combine the series or dataframe...&#39;compute.ops_on_diff_frames&#39; option.</span>
<span class="sd"> Enable &#39;compute.ops_on_diff_frames&#39; to combine Spark DataFrame and pandas-on-Spark index</span>
<span class="sd"> &gt;&gt;&gt; with ps.option_context(&quot;compute.ops_on_diff_frames&quot;, True):</span>
<span class="sd"> ... ps.DataFrame(data=sdf, index=ps.Index([0, 1, 2]))</span>
<span class="sd"> x y</span>
<span class="sd"> 0 Data 1.0</span>
<span class="sd"> 1 Bricks 2.0</span>
<span class="sd"> 2 None NaN</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span> <span class="c1"># type: ignore[no-untyped-def]</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">data</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">copy</span><span class="o">=</span><span class="kc">False</span>
<span class="p">):</span>
<span class="n">SparkDataFrame</span> <span class="o">=</span> <span class="n">get_dataframe_class</span><span class="p">()</span>
<span class="n">index_assigned</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">data</span><span class="p">,</span> <span class="n">InternalFrame</span><span class="p">):</span>
<span class="k">assert</span> <span class="n">columns</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">assert</span> <span class="n">dtype</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">copy</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">data</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">data</span><span class="p">,</span> <span class="n">SparkDataFrame</span><span class="p">):</span>
<span class="k">assert</span> <span class="n">columns</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">assert</span> <span class="n">dtype</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">copy</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span><span class="n">spark_frame</span><span class="o">=</span><span class="n">data</span><span class="p">,</span> <span class="n">index_spark_columns</span><span class="o">=</span><span class="kc">None</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">data</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">):</span>
<span class="k">assert</span> <span class="n">columns</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">assert</span> <span class="n">dtype</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">copy</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">data</span><span class="o">.</span><span class="n">_internal</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">data</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="k">assert</span> <span class="n">dtype</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">copy</span>
<span class="c1"># For pandas compatibility when `columns` contains only one valid column.</span>
<span class="k">if</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="p">(</span><span class="nb">dict</span><span class="p">,</span> <span class="nb">list</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">))</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span>
<span class="n">columns</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">columns</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span> <span class="k">else</span> <span class="n">columns</span>
<span class="k">assert</span> <span class="n">columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">==</span> <span class="n">data</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">data</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span><span class="o">.</span><span class="n">_internal</span>
<span class="k">else</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.indexes.base</span> <span class="kn">import</span> <span class="n">Index</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="n">Index</span><span class="p">):</span>
<span class="c1"># with local data, collect ps.Index to driver</span>
<span class="c1"># to avoid mismatched results between</span>
<span class="c1"># ps.DataFrame([1, 2], index=ps.Index([1, 2]))</span>
<span class="c1"># and</span>
<span class="c1"># pd.DataFrame([1, 2], index=pd.Index([1, 2]))</span>
<span class="n">index</span> <span class="o">=</span> <span class="n">index</span><span class="o">.</span><span class="n">_to_pandas</span><span class="p">()</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">data</span><span class="o">=</span><span class="n">data</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="n">index</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="n">dtype</span><span class="p">,</span> <span class="n">copy</span><span class="o">=</span><span class="n">copy</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="o">.</span><span class="n">from_pandas</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span>
<span class="n">index_assigned</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">index_assigned</span><span class="p">:</span>
<span class="c1"># TODO(SPARK-40226): Support MultiIndex</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="p">(</span><span class="n">ps</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Cannot combine a Distributed Dataset with a MultiIndex&quot;</span><span class="p">)</span>
<span class="n">data_df</span> <span class="o">=</span> <span class="n">ps</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">data</span><span class="o">=</span><span class="n">data</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="n">dtype</span><span class="p">,</span> <span class="n">copy</span><span class="o">=</span><span class="n">copy</span><span class="p">)</span>
<span class="n">index_ps</span> <span class="o">=</span> <span class="n">ps</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span><span class="n">index</span><span class="p">)</span>
<span class="n">index_df</span> <span class="o">=</span> <span class="n">index_ps</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="k">if</span> <span class="n">same_anchor</span><span class="p">(</span><span class="n">data_df</span><span class="p">,</span> <span class="n">index_df</span><span class="p">):</span>
<span class="n">data_labels</span> <span class="o">=</span> <span class="n">data_df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">data_pssers</span> <span class="o">=</span> <span class="p">[</span><span class="n">data_df</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">data_labels</span><span class="p">]</span>
<span class="n">index_labels</span> <span class="o">=</span> <span class="n">index_df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">index_pssers</span> <span class="o">=</span> <span class="p">[</span><span class="n">index_df</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">index_labels</span><span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">data_df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">data_pssers</span> <span class="o">+</span> <span class="n">index_pssers</span><span class="p">)</span>
<span class="n">combined</span> <span class="o">=</span> <span class="n">ps</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">index_labels</span><span class="p">)</span>
<span class="n">combined</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="n">index_ps</span><span class="o">.</span><span class="n">name</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># drop un-matched rows in `data`</span>
<span class="c1"># note that `combine_frames` cannot work with a MultiIndex for now</span>
<span class="n">combined</span> <span class="o">=</span> <span class="n">combine_frames</span><span class="p">(</span><span class="n">data_df</span><span class="p">,</span> <span class="n">index_df</span><span class="p">,</span> <span class="n">how</span><span class="o">=</span><span class="s2">&quot;right&quot;</span><span class="p">)</span>
<span class="n">combined_labels</span> <span class="o">=</span> <span class="n">combined</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">index_labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">label</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">combined_labels</span> <span class="k">if</span> <span class="n">label</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">==</span> <span class="s2">&quot;that&quot;</span><span class="p">]</span>
<span class="n">combined</span> <span class="o">=</span> <span class="n">combined</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">index_labels</span><span class="p">)</span>
<span class="n">combined</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">_column_labels</span> <span class="o">=</span> <span class="n">data_df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">combined</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">_column_label_names</span> <span class="o">=</span> <span class="n">data_df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">_column_label_names</span>
<span class="n">combined</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">_index_names</span> <span class="o">=</span> <span class="n">index_df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">combined</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="n">index_ps</span><span class="o">.</span><span class="n">name</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">combined</span><span class="o">.</span><span class="n">_internal</span>
<span class="nb">object</span><span class="o">.</span><span class="fm">__setattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_internal_frame&quot;</span><span class="p">,</span> <span class="n">internal</span><span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_pssers</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="n">Label</span><span class="p">,</span> <span class="s2">&quot;Series&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Return a dict of column label -&gt; Series which anchors `self`.&quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_psseries&quot;</span><span class="p">):</span>
<span class="nb">object</span><span class="o">.</span><span class="fm">__setattr__</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="s2">&quot;_psseries&quot;</span><span class="p">,</span>
<span class="p">{</span><span class="n">label</span><span class="p">:</span> <span class="n">Series</span><span class="p">(</span><span class="n">data</span><span class="o">=</span><span class="bp">self</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">},</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">psseries</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">Dict</span><span class="p">[</span><span class="n">Label</span><span class="p">,</span> <span class="n">Series</span><span class="p">],</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psseries</span><span class="p">)</span> <span class="c1"># type: ignore[has-type]</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">==</span> <span class="nb">len</span><span class="p">(</span><span class="n">psseries</span><span class="p">),</span> <span class="p">(</span>
<span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">),</span>
<span class="nb">len</span><span class="p">(</span><span class="n">psseries</span><span class="p">),</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="bp">self</span> <span class="ow">is</span> <span class="ow">not</span> <span class="n">psser</span><span class="o">.</span><span class="n">_psdf</span> <span class="k">for</span> <span class="n">psser</span> <span class="ow">in</span> <span class="n">psseries</span><span class="o">.</span><span class="n">values</span><span class="p">()):</span>
<span class="c1"># Refresh the dict to contain only Series anchoring `self`.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_psseries</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">label</span><span class="p">:</span> <span class="p">(</span>
<span class="n">psseries</span><span class="p">[</span><span class="n">label</span><span class="p">]</span>
<span class="k">if</span> <span class="bp">self</span> <span class="ow">is</span> <span class="n">psseries</span><span class="p">[</span><span class="n">label</span><span class="p">]</span><span class="o">.</span><span class="n">_psdf</span>
<span class="k">else</span> <span class="n">Series</span><span class="p">(</span><span class="n">data</span><span class="o">=</span><span class="bp">self</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="n">label</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">}</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psseries</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_internal</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">InternalFrame</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span><span class="n">InternalFrame</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal_frame</span><span class="p">)</span> <span class="c1"># type: ignore[has-type]</span>
<span class="k">def</span> <span class="nf">_update_internal_frame</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">internal</span><span class="p">:</span> <span class="n">InternalFrame</span><span class="p">,</span>
<span class="n">check_same_anchor</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">anchor_force_disconnect</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</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;</span>
<span class="sd"> Update InternalFrame with the given one.</span>
<span class="sd"> If the column_label is changed or the new InternalFrame is not the same `anchor` or the</span>
<span class="sd"> `anchor_force_disconnect` flag is set to True, disconnect the original anchor and create</span>
<span class="sd"> a new one.</span>
<span class="sd"> If `check_same_anchor` is `False`, checking whether the same anchor is ignored</span>
<span class="sd"> and force to update the InternalFrame, e.g., replacing the internal with the resolved_copy,</span>
<span class="sd"> updating the underlying Spark DataFrame which need to combine a different Spark DataFrame.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> internal : InternalFrame</span>
<span class="sd"> The new InternalFrame</span>
<span class="sd"> check_same_anchor : bool</span>
<span class="sd"> Whether checking the same anchor</span>
<span class="sd"> anchor_force_disconnect : bool</span>
<span class="sd"> Force to disconnect the original anchor and create a new one</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_psseries&quot;</span><span class="p">):</span>
<span class="n">psseries</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">old_label</span><span class="p">,</span> <span class="n">new_label</span> <span class="ow">in</span> <span class="n">zip_longest</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">):</span>
<span class="k">if</span> <span class="n">old_label</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pssers</span><span class="p">[</span><span class="n">old_label</span><span class="p">]</span>
<span class="n">renamed</span> <span class="o">=</span> <span class="n">old_label</span> <span class="o">!=</span> <span class="n">new_label</span>
<span class="n">not_same_anchor</span> <span class="o">=</span> <span class="n">check_same_anchor</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">same_anchor</span><span class="p">(</span><span class="n">internal</span><span class="p">,</span> <span class="n">psser</span><span class="p">)</span>
<span class="k">if</span> <span class="n">renamed</span> <span class="ow">or</span> <span class="n">not_same_anchor</span> <span class="ow">or</span> <span class="n">anchor_force_disconnect</span><span class="p">:</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">select_column</span><span class="p">(</span><span class="n">old_label</span><span class="p">))</span>
<span class="n">psser</span><span class="o">.</span><span class="n">_update_anchor</span><span class="p">(</span><span class="n">psdf</span><span class="p">)</span>
<span class="n">psser</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">new_label</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">psser</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="n">Series</span><span class="p">(</span><span class="n">data</span><span class="o">=</span><span class="bp">self</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="n">new_label</span><span class="p">)</span>
<span class="n">psseries</span><span class="p">[</span><span class="n">new_label</span><span class="p">]</span> <span class="o">=</span> <span class="n">psser</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_psseries</span> <span class="o">=</span> <span class="n">psseries</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal_frame</span> <span class="o">=</span> <span class="n">internal</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_repr_pandas_cache&quot;</span><span class="p">):</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_repr_pandas_cache</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">ndim</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">int</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return an int representing the number of array dimensions.</span>
<span class="sd"> return 2 for DataFrame.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[1, 2], [4, 5], [7, 8]],</span>
<span class="sd"> ... index=[&#39;cobra&#39;, &#39;viper&#39;, None],</span>
<span class="sd"> ... columns=[&#39;max_speed&#39;, &#39;shield&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +SKIP</span>
<span class="sd"> max_speed shield</span>
<span class="sd"> cobra 1 2</span>
<span class="sd"> viper 4 5</span>
<span class="sd"> None 7 8</span>
<span class="sd"> &gt;&gt;&gt; df.ndim</span>
<span class="sd"> 2</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="mi">2</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">axes</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">List</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a list representing the axes of the DataFrame.</span>
<span class="sd"> It has the row axis labels and column axis labels as the only members.</span>
<span class="sd"> They are returned in that order.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;col1&#39;: [1, 2], &#39;col2&#39;: [3, 4]})</span>
<span class="sd"> &gt;&gt;&gt; df.axes</span>
<span class="sd"> [Index([0, 1], dtype=&#39;int64&#39;), Index([&#39;col1&#39;, &#39;col2&#39;], dtype=&#39;object&#39;)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">_reduce_for_stat_function</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">sfun</span><span class="p">:</span> <span class="n">Callable</span><span class="p">[[</span><span class="s2">&quot;Series&quot;</span><span class="p">],</span> <span class="n">PySparkColumn</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="n">axis</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Axis</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">numeric_only</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">skipna</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Applies sfun to each column and returns a pd.Series where the number of rows equals the</span>
<span class="sd"> number of columns.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> sfun : either an 1-arg function that takes a Column and returns a Column, or</span>
<span class="sd"> a 2-arg function that takes a Column and its DataType and returns a Column.</span>
<span class="sd"> axis: used only for sanity check because the series only supports index axis.</span>
<span class="sd"> name : original pandas API name.</span>
<span class="sd"> axis : axis to apply. 0 or 1, or &#39;index&#39; or &#39;columns.</span>
<span class="sd"> numeric_only : bool, default False</span>
<span class="sd"> Include only float, int, boolean columns.</span>
<span class="sd"> skipna : bool, default True</span>
<span class="sd"> Exclude NA/null values when computing the result.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span><span class="p">,</span> <span class="n">first_series</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">min_count</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;min_count&quot;</span><span class="p">,</span> <span class="mi">0</span><span class="p">)</span>
<span class="n">exprs</span> <span class="o">=</span> <span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">StringType</span><span class="p">())</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">)]</span>
<span class="n">new_column_labels</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">is_numeric_or_boolean</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">keep_column</span> <span class="o">=</span> <span class="ow">not</span> <span class="n">numeric_only</span> <span class="ow">or</span> <span class="n">is_numeric_or_boolean</span>
<span class="k">if</span> <span class="n">keep_column</span><span class="p">:</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">skipna</span> <span class="ow">and</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.eager_check&quot;</span><span class="p">)</span> <span class="ow">and</span> <span class="n">psser</span><span class="o">.</span><span class="n">hasnans</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">first</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">nan</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">sfun</span><span class="p">(</span><span class="n">psser</span><span class="p">)</span>
<span class="k">if</span> <span class="n">min_count</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">Frame</span><span class="o">.</span><span class="n">_count_expr</span><span class="p">(</span><span class="n">psser</span><span class="p">)</span> <span class="o">&gt;=</span> <span class="n">min_count</span><span class="p">,</span> <span class="n">scol</span><span class="p">)</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">)))</span>
<span class="n">new_column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">exprs</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">return</span> <span class="n">Series</span><span class="p">([],</span> <span class="n">dtype</span><span class="o">=</span><span class="s2">&quot;float64&quot;</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">exprs</span><span class="p">)</span>
<span class="c1"># The data is expected to be small so it&#39;s fine to transpose/use the default index.</span>
<span class="k">with</span> <span class="n">ps</span><span class="o">.</span><span class="n">option_context</span><span class="p">(</span><span class="s2">&quot;compute.max_rows&quot;</span><span class="p">,</span> <span class="mi">1</span><span class="p">):</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">)],</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">new_column_labels</span><span class="p">,</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span><span class="o">.</span><span class="n">transpose</span><span class="p">())</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Here we execute with the first 1000 to get the return type.</span>
<span class="c1"># If the records were less than 1000, it uses pandas API directly for a shortcut.</span>
<span class="n">limit</span> <span class="o">=</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.shortcut_limit&quot;</span><span class="p">)</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">limit</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span>
<span class="n">pser</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">pdf</span><span class="p">,</span> <span class="n">name</span><span class="p">)(</span><span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">numeric_only</span><span class="o">=</span><span class="n">numeric_only</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="n">limit</span><span class="p">:</span>
<span class="k">return</span> <span class="n">Series</span><span class="p">(</span><span class="n">pser</span><span class="p">)</span>
<span class="nd">@pandas_udf</span><span class="p">(</span><span class="n">returnType</span><span class="o">=</span><span class="n">as_spark_type</span><span class="p">(</span><span class="n">pser</span><span class="o">.</span><span class="n">dtype</span><span class="o">.</span><span class="n">type</span><span class="p">))</span> <span class="c1"># type: ignore[call-overload]</span>
<span class="k">def</span> <span class="nf">calculate_columns_axis</span><span class="p">(</span><span class="o">*</span><span class="n">cols</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">concat</span><span class="p">(</span><span class="n">cols</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="mi">1</span><span class="p">),</span> <span class="n">name</span><span class="p">)(</span>
<span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">numeric_only</span><span class="o">=</span><span class="n">numeric_only</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span>
<span class="p">)</span>
<span class="n">column_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">),</span>
<span class="s2">&quot;__calculate_columns_axis__&quot;</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="o">+</span> <span class="p">[</span><span class="n">calculate_columns_axis</span><span class="p">(</span><span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">column_name</span><span class="p">)]</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">))</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">pser</span><span class="o">.</span><span class="n">name</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_psser_for</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">label</span><span class="p">:</span> <span class="n">Label</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Create Series with a proper column label.</span>
<span class="sd"> The given label must be verified to exist in `InternalFrame.column_labels`.</span>
<span class="sd"> For example, in some method, self is like:</span>
<span class="sd"> &gt;&gt;&gt; self = ps.range(3)</span>
<span class="sd"> `self._psser_for(label)` can be used with `InternalFrame.column_labels`:</span>
<span class="sd"> &gt;&gt;&gt; self._psser_for(self._internal.column_labels[0])</span>
<span class="sd"> 0 0</span>
<span class="sd"> 1 1</span>
<span class="sd"> 2 2</span>
<span class="sd"> Name: id, dtype: int64</span>
<span class="sd"> `self._psser_for(label)` must not be used directly with user inputs.</span>
<span class="sd"> In that case, `self[label]` should be used instead, which checks the label exists or not:</span>
<span class="sd"> &gt;&gt;&gt; self[&#39;id&#39;]</span>
<span class="sd"> 0 0</span>
<span class="sd"> 1 1</span>
<span class="sd"> 2 2</span>
<span class="sd"> Name: id, dtype: int64</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pssers</span><span class="p">[</span><span class="n">label</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">_apply_series_op</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">op</span><span class="p">:</span> <span class="n">Callable</span><span class="p">[[</span><span class="s2">&quot;Series&quot;</span><span class="p">],</span> <span class="n">Union</span><span class="p">[</span><span class="s2">&quot;Series&quot;</span><span class="p">,</span> <span class="n">PySparkColumn</span><span class="p">]],</span>
<span class="n">should_resolve</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="n">applied</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">op</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)))</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">applied</span><span class="p">)</span>
<span class="k">if</span> <span class="n">should_resolve</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="c1"># Arithmetic Operators</span>
<span class="k">def</span> <span class="nf">_map_series_op</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">op</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.base</span> <span class="kn">import</span> <span class="n">IndexOpsMixin</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">)</span> <span class="ow">and</span> <span class="p">(</span>
<span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">IndexOpsMixin</span><span class="p">)</span> <span class="ow">or</span> <span class="n">is_sequence</span><span class="p">(</span><span class="n">other</span><span class="p">)</span>
<span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;</span><span class="si">%s</span><span class="s2"> with a sequence is currently not supported; &quot;</span>
<span class="s2">&quot;however, got </span><span class="si">%s</span><span class="s2">.&quot;</span> <span class="o">%</span> <span class="p">(</span><span class="n">op</span><span class="p">,</span> <span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">!=</span> <span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;cannot join with no overlapping index names&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">same_anchor</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="c1"># Different DataFrames</span>
<span class="k">def</span> <span class="nf">apply_op</span><span class="p">(</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span><span class="p">,</span>
<span class="n">this_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">],</span>
<span class="n">that_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">],</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterator</span><span class="p">[</span><span class="n">Tuple</span><span class="p">[</span><span class="s2">&quot;Series&quot;</span><span class="p">,</span> <span class="n">Label</span><span class="p">]]:</span>
<span class="k">for</span> <span class="n">this_label</span><span class="p">,</span> <span class="n">that_label</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">this_column_labels</span><span class="p">,</span> <span class="n">that_column_labels</span><span class="p">):</span>
<span class="k">yield</span> <span class="p">(</span>
<span class="nb">getattr</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">this_label</span><span class="p">),</span> <span class="n">op</span><span class="p">)(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">that_label</span><span class="p">)</span>
<span class="p">)</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">this_label</span><span class="p">),</span>
<span class="n">this_label</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">align_diff_frames</span><span class="p">(</span><span class="n">apply_op</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">fillna</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">how</span><span class="o">=</span><span class="s2">&quot;full&quot;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">applied</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">),</span> <span class="n">op</span><span class="p">)(</span><span class="n">other</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span>
<span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">:</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span>
<span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">applied</span><span class="p">,</span> <span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">psser</span><span class="p">,</span> <span class="n">op</span><span class="p">)(</span><span class="n">other</span><span class="p">))</span>
<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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;add&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__radd__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;radd&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__truediv__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;truediv&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__rtruediv__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;rtruediv&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__mul__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;mul&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__rmul__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;rmul&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__sub__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;sub&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__rsub__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;rsub&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__pow__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;pow&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__rpow__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;rpow&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__mod__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;mod&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__rmod__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;rmod&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__floordiv__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;floordiv&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__rfloordiv__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;rfloordiv&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__abs__</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="nb">abs</span><span class="p">(</span><span class="n">psser</span><span class="p">))</span>
<span class="k">def</span> <span class="fm">__neg__</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="o">-</span><span class="n">psser</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.add"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.add.html#pyspark.pandas.DataFrame.add">[docs]</a> <span class="k">def</span> <span class="nf">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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">+</span> <span class="n">other</span></div>
<span class="c1"># create accessor for plot</span>
<span class="n">plot</span> <span class="o">=</span> <span class="n">CachedAccessor</span><span class="p">(</span><span class="s2">&quot;plot&quot;</span><span class="p">,</span> <span class="n">PandasOnSparkPlotAccessor</span><span class="p">)</span>
<span class="c1"># create accessor for Spark related methods.</span>
<span class="n">spark</span> <span class="o">=</span> <span class="n">CachedAccessor</span><span class="p">(</span><span class="s2">&quot;spark&quot;</span><span class="p">,</span> <span class="n">SparkFrameMethods</span><span class="p">)</span>
<span class="c1"># create accessor for pandas-on-Spark specific methods.</span>
<span class="n">pandas_on_spark</span> <span class="o">=</span> <span class="n">CachedAccessor</span><span class="p">(</span><span class="s2">&quot;pandas_on_spark&quot;</span><span class="p">,</span> <span class="n">PandasOnSparkFrameMethods</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.hist"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.hist.html#pyspark.pandas.DataFrame.hist">[docs]</a> <span class="nd">@no_type_check</span>
<span class="k">def</span> <span class="nf">hist</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">bins</span><span class="o">=</span><span class="mi">10</span><span class="p">,</span> <span class="o">**</span><span class="n">kwds</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">plot</span><span class="o">.</span><span class="n">hist</span><span class="p">(</span><span class="n">bins</span><span class="p">,</span> <span class="o">**</span><span class="n">kwds</span><span class="p">)</span></div>
<span class="n">hist</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">PandasOnSparkPlotAccessor</span><span class="o">.</span><span class="n">hist</span><span class="o">.</span><span class="vm">__doc__</span>
<div class="viewcode-block" id="DataFrame.boxplot"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.boxplot.html#pyspark.pandas.DataFrame.boxplot">[docs]</a> <span class="nd">@no_type_check</span>
<span class="k">def</span> <span class="nf">boxplot</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwds</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">plot</span><span class="o">.</span><span class="n">box</span><span class="p">(</span><span class="o">**</span><span class="n">kwds</span><span class="p">)</span></div>
<span class="n">boxplot</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">PandasOnSparkPlotAccessor</span><span class="o">.</span><span class="n">box</span><span class="o">.</span><span class="vm">__doc__</span>
<div class="viewcode-block" id="DataFrame.kde"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.kde.html#pyspark.pandas.DataFrame.kde">[docs]</a> <span class="nd">@no_type_check</span>
<span class="k">def</span> <span class="nf">kde</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">bw_method</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">ind</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="o">**</span><span class="n">kwds</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">plot</span><span class="o">.</span><span class="n">kde</span><span class="p">(</span><span class="n">bw_method</span><span class="p">,</span> <span class="n">ind</span><span class="p">,</span> <span class="o">**</span><span class="n">kwds</span><span class="p">)</span></div>
<span class="n">kde</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">PandasOnSparkPlotAccessor</span><span class="o">.</span><span class="n">kde</span><span class="o">.</span><span class="vm">__doc__</span>
<span class="n">add</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Addition&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;+&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;dataframe + other&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;radd&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.radd"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.radd.html#pyspark.pandas.DataFrame.radd">[docs]</a> <span class="k">def</span> <span class="nf">radd</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">other</span> <span class="o">+</span> <span class="bp">self</span></div>
<span class="n">radd</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Addition&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;+&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;other + dataframe&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;add&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.div"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.div.html#pyspark.pandas.DataFrame.div">[docs]</a> <span class="k">def</span> <span class="nf">div</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">/</span> <span class="n">other</span></div>
<span class="n">div</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Floating division&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;/&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;dataframe / other&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;rdiv&quot;</span>
<span class="p">)</span>
<span class="n">divide</span> <span class="o">=</span> <span class="n">div</span>
<div class="viewcode-block" id="DataFrame.rdiv"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rdiv.html#pyspark.pandas.DataFrame.rdiv">[docs]</a> <span class="k">def</span> <span class="nf">rdiv</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">other</span> <span class="o">/</span> <span class="bp">self</span></div>
<span class="n">rdiv</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Floating division&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;/&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;other / dataframe&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;div&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.truediv"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.truediv.html#pyspark.pandas.DataFrame.truediv">[docs]</a> <span class="k">def</span> <span class="nf">truediv</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">/</span> <span class="n">other</span></div>
<span class="n">truediv</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Floating division&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;/&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;dataframe / other&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;rtruediv&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.rtruediv"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rtruediv.html#pyspark.pandas.DataFrame.rtruediv">[docs]</a> <span class="k">def</span> <span class="nf">rtruediv</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">other</span> <span class="o">/</span> <span class="bp">self</span></div>
<span class="n">rtruediv</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Floating division&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;/&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;other / dataframe&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;truediv&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.mul"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.mul.html#pyspark.pandas.DataFrame.mul">[docs]</a> <span class="k">def</span> <span class="nf">mul</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">*</span> <span class="n">other</span></div>
<span class="n">mul</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Multiplication&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;*&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;dataframe * other&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;rmul&quot;</span>
<span class="p">)</span>
<span class="n">multiply</span> <span class="o">=</span> <span class="n">mul</span>
<div class="viewcode-block" id="DataFrame.rmul"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rmul.html#pyspark.pandas.DataFrame.rmul">[docs]</a> <span class="k">def</span> <span class="nf">rmul</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">other</span> <span class="o">*</span> <span class="bp">self</span></div>
<span class="n">rmul</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Multiplication&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;*&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;other * dataframe&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;mul&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.sub"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.sub.html#pyspark.pandas.DataFrame.sub">[docs]</a> <span class="k">def</span> <span class="nf">sub</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">-</span> <span class="n">other</span></div>
<span class="n">sub</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Subtraction&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;-&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;dataframe - other&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;rsub&quot;</span>
<span class="p">)</span>
<span class="n">subtract</span> <span class="o">=</span> <span class="n">sub</span>
<div class="viewcode-block" id="DataFrame.rsub"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rsub.html#pyspark.pandas.DataFrame.rsub">[docs]</a> <span class="k">def</span> <span class="nf">rsub</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">other</span> <span class="o">-</span> <span class="bp">self</span></div>
<span class="n">rsub</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Subtraction&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;-&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;other - dataframe&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;sub&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.mod"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.mod.html#pyspark.pandas.DataFrame.mod">[docs]</a> <span class="k">def</span> <span class="nf">mod</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">%</span> <span class="n">other</span></div>
<span class="n">mod</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Modulo&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;%&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;dataframe </span><span class="si">% o</span><span class="s2">ther&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;rmod&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.rmod"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rmod.html#pyspark.pandas.DataFrame.rmod">[docs]</a> <span class="k">def</span> <span class="nf">rmod</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">other</span> <span class="o">%</span> <span class="bp">self</span></div>
<span class="n">rmod</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Modulo&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;%&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;other </span><span class="si">% d</span><span class="s2">ataframe&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;mod&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.pow"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.pow.html#pyspark.pandas.DataFrame.pow">[docs]</a> <span class="k">def</span> <span class="nf">pow</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">**</span><span class="n">other</span></div>
<span class="nb">pow</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Exponential power of series&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;**&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;dataframe ** other&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;rpow&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.rpow"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rpow.html#pyspark.pandas.DataFrame.rpow">[docs]</a> <span class="k">def</span> <span class="nf">rpow</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">other</span><span class="o">**</span><span class="bp">self</span></div>
<span class="n">rpow</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Exponential power&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;**&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;other ** dataframe&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;pow&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.floordiv"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.floordiv.html#pyspark.pandas.DataFrame.floordiv">[docs]</a> <span class="k">def</span> <span class="nf">floordiv</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">//</span> <span class="n">other</span></div>
<span class="n">floordiv</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Integer division&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;//&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;dataframe // other&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;rfloordiv&quot;</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.rfloordiv"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rfloordiv.html#pyspark.pandas.DataFrame.rfloordiv">[docs]</a> <span class="k">def</span> <span class="nf">rfloordiv</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">other</span> <span class="o">//</span> <span class="bp">self</span></div>
<span class="n">rfloordiv</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">_flex_doc_FRAME</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">desc</span><span class="o">=</span><span class="s2">&quot;Integer division&quot;</span><span class="p">,</span> <span class="n">op_name</span><span class="o">=</span><span class="s2">&quot;//&quot;</span><span class="p">,</span> <span class="n">equiv</span><span class="o">=</span><span class="s2">&quot;other // dataframe&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="s2">&quot;floordiv&quot;</span>
<span class="p">)</span>
<span class="c1"># Comparison Operators</span>
<span class="k">def</span> <span class="fm">__eq__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span> <span class="c1"># type: ignore[override]</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;eq&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__ne__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span> <span class="c1"># type: ignore[override]</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;ne&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__lt__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;lt&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__le__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;le&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__ge__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;ge&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__gt__</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_map_series_op</span><span class="p">(</span><span class="s2">&quot;gt&quot;</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.eq"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.eq.html#pyspark.pandas.DataFrame.eq">[docs]</a> <span class="k">def</span> <span class="nf">eq</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compare if the current value is equal to the other.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 4],</span>
<span class="sd"> ... &#39;b&#39;: [1, np.nan, 1, np.nan]},</span>
<span class="sd"> ... index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;], columns=[&#39;a&#39;, &#39;b&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.eq(1)</span>
<span class="sd"> a b</span>
<span class="sd"> a True True</span>
<span class="sd"> b False False</span>
<span class="sd"> c False True</span>
<span class="sd"> d False False</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">==</span> <span class="n">other</span></div>
<span class="n">equals</span> <span class="o">=</span> <span class="n">eq</span>
<div class="viewcode-block" id="DataFrame.gt"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.gt.html#pyspark.pandas.DataFrame.gt">[docs]</a> <span class="k">def</span> <span class="nf">gt</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compare if the current value is greater than the other.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 4],</span>
<span class="sd"> ... &#39;b&#39;: [1, np.nan, 1, np.nan]},</span>
<span class="sd"> ... index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;], columns=[&#39;a&#39;, &#39;b&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.gt(2)</span>
<span class="sd"> a b</span>
<span class="sd"> a False False</span>
<span class="sd"> b False False</span>
<span class="sd"> c True False</span>
<span class="sd"> d True False</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">&gt;</span> <span class="n">other</span></div>
<div class="viewcode-block" id="DataFrame.ge"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.ge.html#pyspark.pandas.DataFrame.ge">[docs]</a> <span class="k">def</span> <span class="nf">ge</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compare if the current value is greater than or equal to the other.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 4],</span>
<span class="sd"> ... &#39;b&#39;: [1, np.nan, 1, np.nan]},</span>
<span class="sd"> ... index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;], columns=[&#39;a&#39;, &#39;b&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.ge(1)</span>
<span class="sd"> a b</span>
<span class="sd"> a True True</span>
<span class="sd"> b True False</span>
<span class="sd"> c True True</span>
<span class="sd"> d True False</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">&gt;=</span> <span class="n">other</span></div>
<div class="viewcode-block" id="DataFrame.lt"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.lt.html#pyspark.pandas.DataFrame.lt">[docs]</a> <span class="k">def</span> <span class="nf">lt</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compare if the current value is less than the other.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 4],</span>
<span class="sd"> ... &#39;b&#39;: [1, np.nan, 1, np.nan]},</span>
<span class="sd"> ... index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;], columns=[&#39;a&#39;, &#39;b&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.lt(1)</span>
<span class="sd"> a b</span>
<span class="sd"> a False False</span>
<span class="sd"> b False False</span>
<span class="sd"> c False False</span>
<span class="sd"> d False False</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">&lt;</span> <span class="n">other</span></div>
<div class="viewcode-block" id="DataFrame.le"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.le.html#pyspark.pandas.DataFrame.le">[docs]</a> <span class="k">def</span> <span class="nf">le</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compare if the current value is less than or equal to the other.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 4],</span>
<span class="sd"> ... &#39;b&#39;: [1, np.nan, 1, np.nan]},</span>
<span class="sd"> ... index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;], columns=[&#39;a&#39;, &#39;b&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.le(2)</span>
<span class="sd"> a b</span>
<span class="sd"> a True True</span>
<span class="sd"> b True False</span>
<span class="sd"> c False True</span>
<span class="sd"> d False False</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">&lt;=</span> <span class="n">other</span></div>
<div class="viewcode-block" id="DataFrame.ne"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.ne.html#pyspark.pandas.DataFrame.ne">[docs]</a> <span class="k">def</span> <span class="nf">ne</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">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compare if the current value is not equal to the other.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 4],</span>
<span class="sd"> ... &#39;b&#39;: [1, np.nan, 1, np.nan]},</span>
<span class="sd"> ... index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;], columns=[&#39;a&#39;, &#39;b&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.ne(1)</span>
<span class="sd"> a b</span>
<span class="sd"> a False False</span>
<span class="sd"> b True True</span>
<span class="sd"> c True False</span>
<span class="sd"> d True True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span> <span class="o">!=</span> <span class="n">other</span></div>
<div class="viewcode-block" id="DataFrame.applymap"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.applymap.html#pyspark.pandas.DataFrame.applymap">[docs]</a> <span class="k">def</span> <span class="nf">applymap</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">:</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Any</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Apply a function to a Dataframe elementwise.</span>
<span class="sd"> This method applies a function that accepts and returns a scalar</span>
<span class="sd"> to every element of a DataFrame.</span>
<span class="sd"> .. deprecated:: 4.0.0</span>
<span class="sd"> .. note:: this API executes the function once to infer the type which is</span>
<span class="sd"> potentially expensive, for instance, when the dataset is created after</span>
<span class="sd"> aggregations or sorting.</span>
<span class="sd"> To avoid this, specify return type in ``func``, for instance, as below:</span>
<span class="sd"> &gt;&gt;&gt; def square(x) -&gt; np.int32:</span>
<span class="sd"> ... return x ** 2</span>
<span class="sd"> pandas-on-Spark uses return type hints and does not try to infer the type.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> func : callable</span>
<span class="sd"> Python function returns a single value from a single value.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Transformed DataFrame.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[1, 2.12], [3.356, 4.567]])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 1.000 2.120</span>
<span class="sd"> 1 3.356 4.567</span>
<span class="sd"> &gt;&gt;&gt; def str_len(x) -&gt; int:</span>
<span class="sd"> ... return len(str(x))</span>
<span class="sd"> &gt;&gt;&gt; df.applymap(str_len)</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 3 4</span>
<span class="sd"> 1 5 5</span>
<span class="sd"> &gt;&gt;&gt; def power(x) -&gt; float:</span>
<span class="sd"> ... return x ** 2</span>
<span class="sd"> &gt;&gt;&gt; df.applymap(power)</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 1.000000 4.494400</span>
<span class="sd"> 1 11.262736 20.857489</span>
<span class="sd"> You can omit type hints and let pandas-on-Spark infer its type.</span>
<span class="sd"> &gt;&gt;&gt; df.applymap(lambda x: x ** 2)</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 1.000000 4.494400</span>
<span class="sd"> 1 11.262736 20.857489</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;DataFrame.applymap has been deprecated. Use DataFrame.map instead&quot;</span><span class="p">,</span> <span class="ne">FutureWarning</span>
<span class="p">)</span>
<span class="c1"># TODO: We can implement shortcut theoretically since it creates new DataFrame</span>
<span class="c1"># anyway and we don&#39;t have to worry about operations on different DataFrames.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="n">func</span><span class="o">=</span><span class="n">func</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.map"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.map.html#pyspark.pandas.DataFrame.map">[docs]</a> <span class="k">def</span> <span class="nf">map</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">:</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Any</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Apply a function to a Dataframe elementwise.</span>
<span class="sd"> This method applies a function that accepts and returns a scalar</span>
<span class="sd"> to every element of a DataFrame.</span>
<span class="sd"> .. versionadded:: 4.0.0</span>
<span class="sd"> DataFrame.applymap was deprecated and renamed to DataFrame.map.</span>
<span class="sd"> .. note:: this API executes the function once to infer the type which is</span>
<span class="sd"> potentially expensive, for instance, when the dataset is created after</span>
<span class="sd"> aggregations or sorting.</span>
<span class="sd"> To avoid this, specify return type in ``func``, for instance, as below:</span>
<span class="sd"> &gt;&gt;&gt; def square(x) -&gt; np.int32:</span>
<span class="sd"> ... return x ** 2</span>
<span class="sd"> pandas-on-Spark uses return type hints and does not try to infer the type.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> func : callable</span>
<span class="sd"> Python function returns a single value from a single value.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Transformed DataFrame.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[1, 2.12], [3.356, 4.567]])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 1.000 2.120</span>
<span class="sd"> 1 3.356 4.567</span>
<span class="sd"> &gt;&gt;&gt; def str_len(x) -&gt; int:</span>
<span class="sd"> ... return len(str(x))</span>
<span class="sd"> &gt;&gt;&gt; df.map(str_len)</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 3 4</span>
<span class="sd"> 1 5 5</span>
<span class="sd"> &gt;&gt;&gt; def power(x) -&gt; float:</span>
<span class="sd"> ... return x ** 2</span>
<span class="sd"> &gt;&gt;&gt; df.map(power)</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 1.000000 4.494400</span>
<span class="sd"> 1 11.262736 20.857489</span>
<span class="sd"> You can omit type hints and let pandas-on-Spark infer its type.</span>
<span class="sd"> &gt;&gt;&gt; df.map(lambda x: x ** 2)</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 1.000000 4.494400</span>
<span class="sd"> 1 11.262736 20.857489</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># TODO: We can implement shortcut theoretically since it creates new DataFrame</span>
<span class="c1"># anyway and we don&#39;t have to worry about operations on different DataFrames.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">func</span><span class="p">))</span></div>
<span class="c1"># TODO(SPARK-46156): add `axis` parameter.</span>
<div class="viewcode-block" id="DataFrame.aggregate"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.aggregate.html#pyspark.pandas.DataFrame.aggregate">[docs]</a> <span class="k">def</span> <span class="nf">aggregate</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">],</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]])</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Aggregate using one or more operations over the specified axis.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> func : dict or a list</span>
<span class="sd"> a dict mapping from column name (string) to</span>
<span class="sd"> aggregate functions (list of strings).</span>
<span class="sd"> If a list is given, the aggregation is performed against</span>
<span class="sd"> all columns.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> `agg` is an alias for `aggregate`. Use the alias.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.apply : Invoke function on DataFrame.</span>
<span class="sd"> DataFrame.transform : Only perform transforming type operations.</span>
<span class="sd"> DataFrame.groupby : Perform operations over groups.</span>
<span class="sd"> Series.aggregate : The equivalent function for Series.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[1, 2, 3],</span>
<span class="sd"> ... [4, 5, 6],</span>
<span class="sd"> ... [7, 8, 9],</span>
<span class="sd"> ... [np.nan, np.nan, np.nan]],</span>
<span class="sd"> ... columns=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C</span>
<span class="sd"> 0 1.0 2.0 3.0</span>
<span class="sd"> 1 4.0 5.0 6.0</span>
<span class="sd"> 2 7.0 8.0 9.0</span>
<span class="sd"> 3 NaN NaN NaN</span>
<span class="sd"> Aggregate these functions over the rows.</span>
<span class="sd"> &gt;&gt;&gt; df.agg([&#39;sum&#39;, &#39;min&#39;])[[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;]].sort_index()</span>
<span class="sd"> A B C</span>
<span class="sd"> min 1.0 2.0 3.0</span>
<span class="sd"> sum 12.0 15.0 18.0</span>
<span class="sd"> Different aggregations per column.</span>
<span class="sd"> &gt;&gt;&gt; df.agg({&#39;A&#39; : [&#39;sum&#39;, &#39;min&#39;], &#39;B&#39; : [&#39;min&#39;, &#39;max&#39;]})[[&#39;A&#39;, &#39;B&#39;]].sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> max NaN 8.0</span>
<span class="sd"> min 1.0 2.0</span>
<span class="sd"> sum 12.0 NaN</span>
<span class="sd"> For multi-index columns:</span>
<span class="sd"> &gt;&gt;&gt; df.columns = pd.MultiIndex.from_tuples([(&quot;X&quot;, &quot;A&quot;), (&quot;X&quot;, &quot;B&quot;), (&quot;Y&quot;, &quot;C&quot;)])</span>
<span class="sd"> &gt;&gt;&gt; df.agg([&#39;sum&#39;, &#39;min&#39;])[[(&quot;X&quot;, &quot;A&quot;), (&quot;X&quot;, &quot;B&quot;), (&quot;Y&quot;, &quot;C&quot;)]].sort_index()</span>
<span class="sd"> X Y</span>
<span class="sd"> A B C</span>
<span class="sd"> min 1.0 2.0 3.0</span>
<span class="sd"> sum 12.0 15.0 18.0</span>
<span class="sd"> &gt;&gt;&gt; aggregated = df.agg({(&quot;X&quot;, &quot;A&quot;) : [&#39;sum&#39;, &#39;min&#39;], (&quot;X&quot;, &quot;B&quot;) : [&#39;min&#39;, &#39;max&#39;]})</span>
<span class="sd"> &gt;&gt;&gt; aggregated[[(&quot;X&quot;, &quot;A&quot;), (&quot;X&quot;, &quot;B&quot;)]].sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> X</span>
<span class="sd"> A B</span>
<span class="sd"> max NaN 8.0</span>
<span class="sd"> min 1.0 2.0</span>
<span class="sd"> sum 12.0 NaN</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.groupby</span> <span class="kn">import</span> <span class="n">GroupBy</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">all</span><span class="p">((</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">f</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="k">for</span> <span class="n">f</span> <span class="ow">in</span> <span class="n">func</span><span class="p">)):</span>
<span class="n">func</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">([(</span><span class="n">column</span><span class="p">,</span> <span class="n">func</span><span class="p">)</span> <span class="k">for</span> <span class="n">column</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">])</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;If the given function is a list, it &quot;</span>
<span class="s2">&quot;should only contains function names as strings.&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span> <span class="ow">or</span> <span class="ow">not</span> <span class="nb">all</span><span class="p">(</span>
<span class="n">is_name_like_value</span><span class="p">(</span><span class="n">key</span><span class="p">)</span>
<span class="ow">and</span> <span class="p">(</span>
<span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span>
<span class="ow">or</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">value</span><span class="p">))</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">func</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;aggs must be a dict mapping from column name to aggregate &quot;</span>
<span class="s2">&quot;functions (string or list of strings).&quot;</span>
<span class="p">)</span>
<span class="k">with</span> <span class="n">option_context</span><span class="p">(</span><span class="s2">&quot;compute.default_index_type&quot;</span><span class="p">,</span> <span class="s2">&quot;distributed&quot;</span><span class="p">):</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">GroupBy</span><span class="o">.</span><span class="n">_spark_groupby</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">))</span>
<span class="c1"># The codes below basically convert:</span>
<span class="c1">#</span>
<span class="c1"># A B</span>
<span class="c1"># sum min min max</span>
<span class="c1"># 0 12.0 1.0 2.0 8.0</span>
<span class="c1">#</span>
<span class="c1"># to:</span>
<span class="c1"># A B</span>
<span class="c1"># max NaN 8.0</span>
<span class="c1"># min 1.0 2.0</span>
<span class="c1"># sum 12.0 NaN</span>
<span class="c1">#</span>
<span class="c1"># Aggregated output is usually pretty much small.</span>
<span class="k">return</span> <span class="n">psdf</span><span class="o">.</span><span class="n">stack</span><span class="p">()</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="mi">0</span><span class="p">)[</span><span class="nb">list</span><span class="p">(</span><span class="n">func</span><span class="o">.</span><span class="n">keys</span><span class="p">())]</span></div>
<span class="n">agg</span> <span class="o">=</span> <span class="n">aggregate</span>
<div class="viewcode-block" id="DataFrame.corr"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.corr.html#pyspark.pandas.DataFrame.corr">[docs]</a> <span class="k">def</span> <span class="nf">corr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">method</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;pearson&quot;</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute pairwise correlation of columns, excluding NA/null values.</span>
<span class="sd"> .. versionadded:: 3.3.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> method : {&#39;pearson&#39;, &#39;spearman&#39;, &#39;kendall&#39;}</span>
<span class="sd"> * pearson : standard correlation coefficient</span>
<span class="sd"> * spearman : Spearman rank correlation</span>
<span class="sd"> * kendall : Kendall Tau correlation coefficient</span>
<span class="sd"> .. versionchanged:: 3.4.0</span>
<span class="sd"> support &#39;kendall&#39; for method parameter</span>
<span class="sd"> min_periods : int, optional</span>
<span class="sd"> Minimum number of observations required per pair of columns</span>
<span class="sd"> to have a valid result.</span>
<span class="sd"> .. versionadded:: 3.4.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.corrwith</span>
<span class="sd"> Series.corr</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> 1. Pearson, Kendall and Spearman correlation are currently computed using pairwise</span>
<span class="sd"> complete observations.</span>
<span class="sd"> 2. The complexity of Kendall correlation is O(#row * #row), if the dataset is too</span>
<span class="sd"> large, sampling ahead of correlation computation is recommended.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)],</span>
<span class="sd"> ... columns=[&#39;dogs&#39;, &#39;cats&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.corr(&#39;pearson&#39;)</span>
<span class="sd"> dogs cats</span>
<span class="sd"> dogs 1.000000 -0.851064</span>
<span class="sd"> cats -0.851064 1.000000</span>
<span class="sd"> &gt;&gt;&gt; df.corr(&#39;spearman&#39;)</span>
<span class="sd"> dogs cats</span>
<span class="sd"> dogs 1.000000 -0.948683</span>
<span class="sd"> cats -0.948683 1.000000</span>
<span class="sd"> &gt;&gt;&gt; df.corr(&#39;kendall&#39;)</span>
<span class="sd"> dogs cats</span>
<span class="sd"> dogs 1.000000 -0.912871</span>
<span class="sd"> cats -0.912871 1.000000</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">method</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">[</span><span class="s2">&quot;pearson&quot;</span><span class="p">,</span> <span class="s2">&quot;spearman&quot;</span><span class="p">,</span> <span class="s2">&quot;kendall&quot;</span><span class="p">]:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="sa">f</span><span class="s2">&quot;Invalid method </span><span class="si">{</span><span class="n">method</span><span class="si">}</span><span class="s2">&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">min_periods</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">min_periods</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="sa">f</span><span class="s2">&quot;Invalid min_periods type </span><span class="si">{</span><span class="nb">type</span><span class="p">(</span><span class="n">min_periods</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="si">}</span><span class="s2">&quot;</span><span class="p">)</span>
<span class="n">min_periods</span> <span class="o">=</span> <span class="mi">1</span> <span class="k">if</span> <span class="n">min_periods</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="n">min_periods</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="n">numeric_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">label</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">label</span><span class="p">),</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">))</span>
<span class="p">]</span>
<span class="n">numeric_scols</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;double&quot;</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">numeric_labels</span>
<span class="p">]</span>
<span class="n">numeric_col_names</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="p">[</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">numeric_labels</span><span class="p">]</span>
<span class="n">num_scols</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">numeric_scols</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="n">index_1_col_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;__corr_index_1_temp_column__&quot;</span><span class="p">)</span>
<span class="n">index_2_col_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;__corr_index_2_temp_column__&quot;</span><span class="p">)</span>
<span class="c1"># simple dataset</span>
<span class="c1"># +---+---+----+</span>
<span class="c1"># | A| B| C|</span>
<span class="c1"># +---+---+----+</span>
<span class="c1"># | 1| 2| 3.0|</span>
<span class="c1"># | 4| 1|NULL|</span>
<span class="c1"># +---+---+----+</span>
<span class="n">pair_scols</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">num_scols</span><span class="p">):</span>
<span class="k">for</span> <span class="n">j</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="n">num_scols</span><span class="p">):</span>
<span class="n">pair_scols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">i</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">),</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">j</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_2_col_name</span><span class="p">),</span>
<span class="n">numeric_scols</span><span class="p">[</span><span class="n">i</span><span class="p">]</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">CORRELATION_VALUE_1_COLUMN</span><span class="p">),</span>
<span class="n">numeric_scols</span><span class="p">[</span><span class="n">j</span><span class="p">]</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">CORRELATION_VALUE_2_COLUMN</span><span class="p">),</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="c1"># +-------------------+-------------------+-------------------+-------------------+</span>
<span class="c1"># |__tmp_index_1_col__|__tmp_index_2_col__|__tmp_value_1_col__|__tmp_value_2_col__|</span>
<span class="c1"># +-------------------+-------------------+-------------------+-------------------+</span>
<span class="c1"># | 0| 0| 1.0| 1.0|</span>
<span class="c1"># | 0| 1| 1.0| 2.0|</span>
<span class="c1"># | 0| 2| 1.0| 3.0|</span>
<span class="c1"># | 1| 1| 2.0| 2.0|</span>
<span class="c1"># | 1| 2| 2.0| 3.0|</span>
<span class="c1"># | 2| 2| 3.0| 3.0|</span>
<span class="c1"># | 0| 0| 4.0| 4.0|</span>
<span class="c1"># | 0| 1| 4.0| 1.0|</span>
<span class="c1"># | 0| 2| NULL| NULL|</span>
<span class="c1"># | 1| 1| 1.0| 1.0|</span>
<span class="c1"># | 1| 2| NULL| NULL|</span>
<span class="c1"># | 2| 2| NULL| NULL|</span>
<span class="c1"># +-------------------+-------------------+-------------------+-------------------+</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">inline</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span><span class="o">*</span><span class="n">pair_scols</span><span class="p">)))</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">compute</span><span class="p">(</span><span class="n">sdf</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span> <span class="n">groupKeys</span><span class="o">=</span><span class="p">[</span><span class="n">index_1_col_name</span><span class="p">,</span> <span class="n">index_2_col_name</span><span class="p">],</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">)</span>
<span class="k">if</span> <span class="n">method</span> <span class="o">==</span> <span class="s2">&quot;kendall&quot;</span><span class="p">:</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">,</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">)</span> <span class="o">==</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_2_col_name</span><span class="p">),</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="mf">1.0</span><span class="p">))</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">)</span>
<span class="p">),</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">,</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">CORRELATION_COUNT_OUTPUT_COLUMN</span><span class="p">)</span> <span class="o">&lt;</span> <span class="n">min_periods</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">))</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">)</span>
<span class="p">),</span>
<span class="p">)</span>
<span class="c1"># +-------------------+-------------------+----------------+</span>
<span class="c1"># |__tmp_index_1_col__|__tmp_index_2_col__|__tmp_corr_col__|</span>
<span class="c1"># +-------------------+-------------------+----------------+</span>
<span class="c1"># | 2| 2| NULL|</span>
<span class="c1"># | 1| 2| NULL|</span>
<span class="c1"># | 2| 1| NULL|</span>
<span class="c1"># | 1| 1| 1.0|</span>
<span class="c1"># | 0| 0| 1.0|</span>
<span class="c1"># | 0| 1| -1.0|</span>
<span class="c1"># | 1| 0| -1.0|</span>
<span class="c1"># | 0| 2| NULL|</span>
<span class="c1"># | 2| 0| NULL|</span>
<span class="c1"># +-------------------+-------------------+----------------+</span>
<span class="n">auxiliary_col_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;__corr_auxiliary_temp_column__&quot;</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">auxiliary_col_name</span><span class="p">,</span>
<span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">)</span> <span class="o">==</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_2_col_name</span><span class="p">),</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">([</span><span class="mi">0</span><span class="p">]),</span>
<span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">([</span><span class="mi">0</span><span class="p">,</span> <span class="mi">1</span><span class="p">]))</span>
<span class="p">),</span>
<span class="p">)</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">auxiliary_col_name</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">))</span>
<span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_2_col_name</span><span class="p">))</span>
<span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">),</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">auxiliary_col_name</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_2_col_name</span><span class="p">))</span>
<span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">))</span>
<span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_2_col_name</span><span class="p">),</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">),</span>
<span class="p">)</span>
<span class="c1"># +-------------------+--------------------+</span>
<span class="c1"># |__tmp_index_1_col__| __tmp_array_col__|</span>
<span class="c1"># +-------------------+--------------------+</span>
<span class="c1"># | 0|[{0, 1.0}, {1, -1...|</span>
<span class="c1"># | 1|[{0, -1.0}, {1, 1...|</span>
<span class="c1"># | 2|[{0, null}, {1, n...|</span>
<span class="c1"># +-------------------+--------------------+</span>
<span class="n">array_col_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;__corr_array_temp_column__&quot;</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">sdf</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">)</span>
<span class="o">.</span><span class="n">agg</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">array_sort</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">collect_list</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_2_col_name</span><span class="p">),</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">))</span>
<span class="p">)</span>
<span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">array_col_name</span><span class="p">)</span>
<span class="p">)</span>
<span class="o">.</span><span class="n">orderBy</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">num_scols</span><span class="p">):</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="n">auxiliary_col_name</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">array_col_name</span><span class="p">),</span> <span class="n">i</span><span class="p">))</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">numeric_col_names</span><span class="p">[</span><span class="n">i</span><span class="p">],</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="sa">f</span><span class="s2">&quot;</span><span class="si">{</span><span class="n">auxiliary_col_name</span><span class="si">}</span><span class="s2">.</span><span class="si">{</span><span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="si">}</span><span class="s2">&quot;</span><span class="p">),</span>
<span class="p">)</span>
<span class="n">index_col_names</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">if</span> <span class="n">internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">for</span> <span class="n">level</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">):</span>
<span class="n">index_col_name</span> <span class="o">=</span> <span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="n">level</span><span class="p">)</span>
<span class="n">indices</span> <span class="o">=</span> <span class="p">[</span><span class="n">label</span><span class="p">[</span><span class="n">level</span><span class="p">]</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">numeric_labels</span><span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="n">index_col_name</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">indices</span><span class="p">),</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">)))</span>
<span class="n">index_col_names</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">index_col_name</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">,</span>
<span class="n">F</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">numeric_col_names</span><span class="p">),</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">index_1_col_name</span><span class="p">)),</span>
<span class="p">)</span>
<span class="n">index_col_names</span> <span class="o">=</span> <span class="p">[</span><span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">index_col_names</span><span class="p">,</span> <span class="o">*</span><span class="n">numeric_col_names</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">index_col_name</span><span class="p">)</span> <span class="k">for</span> <span class="n">index_col_name</span> <span class="ow">in</span> <span class="n">index_col_names</span>
<span class="p">],</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">numeric_labels</span><span class="p">,</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="n">internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">,</span>
<span class="p">)</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.corrwith"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.corrwith.html#pyspark.pandas.DataFrame.corrwith">[docs]</a> <span class="k">def</span> <span class="nf">corrwith</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="n">DataFrameOrSeries</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="n">drop</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span> <span class="n">method</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;pearson&quot;</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute pairwise correlation.</span>
<span class="sd"> Pairwise correlation is computed between rows or columns of</span>
<span class="sd"> DataFrame with rows or columns of Series or DataFrame. DataFrames</span>
<span class="sd"> are first aligned along both axes before computing the</span>
<span class="sd"> correlations.</span>
<span class="sd"> .. versionadded:: 3.4.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> other : DataFrame, Series</span>
<span class="sd"> Object with which to compute correlations.</span>
<span class="sd"> axis : int, default 0 or &#39;index&#39;</span>
<span class="sd"> Can only be set to 0 now.</span>
<span class="sd"> drop : bool, default False</span>
<span class="sd"> Drop missing indices from result.</span>
<span class="sd"> method : {&#39;pearson&#39;, &#39;spearman&#39;, &#39;kendall&#39;}</span>
<span class="sd"> * pearson : standard correlation coefficient</span>
<span class="sd"> * spearman : Spearman rank correlation</span>
<span class="sd"> * kendall : Kendall Tau correlation coefficient</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series</span>
<span class="sd"> Pairwise correlations.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.corr : Compute pairwise correlation of columns.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame({</span>
<span class="sd"> ... &quot;A&quot;:[1, 5, 7, 8],</span>
<span class="sd"> ... &quot;X&quot;:[5, 8, 4, 3],</span>
<span class="sd"> ... &quot;C&quot;:[10, 4, 9, 3]})</span>
<span class="sd"> &gt;&gt;&gt; df1.corrwith(df1[[&quot;X&quot;, &quot;C&quot;]]).sort_index()</span>
<span class="sd"> A NaN</span>
<span class="sd"> C 1.0</span>
<span class="sd"> X 1.0</span>
<span class="sd"> dtype: float64</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame({</span>
<span class="sd"> ... &quot;A&quot;:[5, 3, 6, 4],</span>
<span class="sd"> ... &quot;B&quot;:[11, 2, 4, 3],</span>
<span class="sd"> ... &quot;C&quot;:[4, 3, 8, 5]})</span>
<span class="sd"> &gt;&gt;&gt; with ps.option_context(&quot;compute.ops_on_diff_frames&quot;, True):</span>
<span class="sd"> ... df1.corrwith(df2).sort_index()</span>
<span class="sd"> A -0.041703</span>
<span class="sd"> B NaN</span>
<span class="sd"> C 0.395437</span>
<span class="sd"> X NaN</span>
<span class="sd"> dtype: float64</span>
<span class="sd"> &gt;&gt;&gt; with ps.option_context(&quot;compute.ops_on_diff_frames&quot;, True):</span>
<span class="sd"> ... df1.corrwith(df2, method=&quot;kendall&quot;).sort_index()</span>
<span class="sd"> A 0.0</span>
<span class="sd"> B NaN</span>
<span class="sd"> C 0.0</span>
<span class="sd"> X NaN</span>
<span class="sd"> dtype: float64</span>
<span class="sd"> &gt;&gt;&gt; with ps.option_context(&quot;compute.ops_on_diff_frames&quot;, True):</span>
<span class="sd"> ... df1.corrwith(df2.B, method=&quot;spearman&quot;).sort_index()</span>
<span class="sd"> A -0.4</span>
<span class="sd"> C 0.8</span>
<span class="sd"> X -0.2</span>
<span class="sd"> dtype: float64</span>
<span class="sd"> &gt;&gt;&gt; with ps.option_context(&quot;compute.ops_on_diff_frames&quot;, True):</span>
<span class="sd"> ... df2.corrwith(df1.X).sort_index()</span>
<span class="sd"> A -0.597614</span>
<span class="sd"> B -0.151186</span>
<span class="sd"> C -0.642857</span>
<span class="sd"> dtype: float64</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span><span class="p">,</span> <span class="n">first_series</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;corrwith currently only works for axis=0&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">method</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">[</span><span class="s2">&quot;pearson&quot;</span><span class="p">,</span> <span class="s2">&quot;spearman&quot;</span><span class="p">,</span> <span class="s2">&quot;kendall&quot;</span><span class="p">]:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="sa">f</span><span class="s2">&quot;Invalid method </span><span class="si">{</span><span class="n">method</span><span class="si">}</span><span class="s2">&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="p">(</span><span class="n">DataFrame</span><span class="p">,</span> <span class="n">Series</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;unsupported type: </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">))</span>
<span class="n">right_is_series</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">Series</span><span class="p">)</span>
<span class="k">if</span> <span class="n">same_anchor</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">combined</span> <span class="o">=</span> <span class="bp">self</span>
<span class="n">this</span> <span class="o">=</span> <span class="bp">self</span>
<span class="n">that</span> <span class="o">=</span> <span class="n">other</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">combined</span> <span class="o">=</span> <span class="n">combine_frames</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">how</span><span class="o">=</span><span class="s2">&quot;inner&quot;</span><span class="p">)</span>
<span class="n">this</span> <span class="o">=</span> <span class="n">combined</span><span class="p">[</span><span class="s2">&quot;this&quot;</span><span class="p">]</span>
<span class="n">that</span> <span class="o">=</span> <span class="n">combined</span><span class="p">[</span><span class="s2">&quot;that&quot;</span><span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">combined</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="n">index_col_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;__corrwith_index_temp_column__&quot;</span><span class="p">)</span>
<span class="n">this_numeric_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">column_label</span> <span class="ow">in</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">),</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">)):</span>
<span class="n">this_numeric_column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span>
<span class="n">that_numeric_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">column_label</span> <span class="ow">in</span> <span class="n">that</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">that</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">),</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">)):</span>
<span class="n">that_numeric_column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span>
<span class="n">intersect_numeric_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">diff_numeric_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">pair_scols</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">if</span> <span class="n">right_is_series</span><span class="p">:</span>
<span class="n">intersect_numeric_column_labels</span> <span class="o">=</span> <span class="n">this_numeric_column_labels</span>
<span class="n">that_scol</span> <span class="o">=</span> <span class="n">that</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">that_numeric_column_labels</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span>
<span class="s2">&quot;double&quot;</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">numeric_column_label</span> <span class="ow">in</span> <span class="n">intersect_numeric_column_labels</span><span class="p">:</span>
<span class="n">this_scol</span> <span class="o">=</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">numeric_column_label</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;double&quot;</span><span class="p">)</span>
<span class="n">pair_scols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">numeric_column_label</span><span class="p">))</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_col_name</span><span class="p">),</span>
<span class="n">this_scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">CORRELATION_VALUE_1_COLUMN</span><span class="p">),</span>
<span class="n">that_scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">CORRELATION_VALUE_2_COLUMN</span><span class="p">),</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">for</span> <span class="n">numeric_column_label</span> <span class="ow">in</span> <span class="n">this_numeric_column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">numeric_column_label</span> <span class="ow">in</span> <span class="n">that_numeric_column_labels</span><span class="p">:</span>
<span class="n">intersect_numeric_column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">numeric_column_label</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">diff_numeric_column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">numeric_column_label</span><span class="p">)</span>
<span class="k">for</span> <span class="n">numeric_column_label</span> <span class="ow">in</span> <span class="n">that_numeric_column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">numeric_column_label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">this_numeric_column_labels</span><span class="p">:</span>
<span class="n">diff_numeric_column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">numeric_column_label</span><span class="p">)</span>
<span class="k">for</span> <span class="n">numeric_column_label</span> <span class="ow">in</span> <span class="n">intersect_numeric_column_labels</span><span class="p">:</span>
<span class="n">this_scol</span> <span class="o">=</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">numeric_column_label</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;double&quot;</span><span class="p">)</span>
<span class="n">that_scol</span> <span class="o">=</span> <span class="n">that</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">numeric_column_label</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;double&quot;</span><span class="p">)</span>
<span class="n">pair_scols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">numeric_column_label</span><span class="p">))</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_col_name</span><span class="p">),</span>
<span class="n">this_scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">CORRELATION_VALUE_1_COLUMN</span><span class="p">),</span>
<span class="n">that_scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">CORRELATION_VALUE_2_COLUMN</span><span class="p">),</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pair_scols</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">inline</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span><span class="o">*</span><span class="n">pair_scols</span><span class="p">)))</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">compute</span><span class="p">(</span><span class="n">sdf</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span> <span class="n">groupKeys</span><span class="o">=</span><span class="p">[</span><span class="n">index_col_name</span><span class="p">],</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">)</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="n">index_col_name</span><span class="p">,</span> <span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;string&quot;</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_col_name</span><span class="p">),</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;double&quot;</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">),</span>
<span class="p">)</span><span class="o">.</span><span class="n">limit</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">drop</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">diff_numeric_column_labels</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">sdf2</span> <span class="o">=</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">([</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">diff_numeric_column_labels</span><span class="p">])</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span>
<span class="n">index_col_name</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="o">.</span><span class="n">limit</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span>
<span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span><span class="n">index_col_name</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_col_name</span><span class="p">))</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">unionByName</span><span class="p">(</span><span class="n">sdf2</span><span class="p">,</span> <span class="n">allowMissingColumns</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">NATURAL_ORDER_COLUMN_NAME</span><span class="p">,</span>
<span class="n">F</span><span class="o">.</span><span class="n">monotonically_increasing_id</span><span class="p">(),</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">index_col_name</span><span class="p">)],</span>
<span class="n">column_labels</span><span class="o">=</span><span class="p">[(</span><span class="n">CORRELATION_CORR_OUTPUT_COLUMN</span><span class="p">,)],</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">sser</span> <span class="o">=</span> <span class="n">first_series</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">))</span>
<span class="n">sser</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">return</span> <span class="n">sser</span></div>
<div class="viewcode-block" id="DataFrame.items"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.items.html#pyspark.pandas.DataFrame.items">[docs]</a> <span class="k">def</span> <span class="nf">items</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterator</span><span class="p">[</span><span class="n">Tuple</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="s2">&quot;Series&quot;</span><span class="p">]]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Iterator over (column name, Series) pairs.</span>
<span class="sd"> Iterates over the DataFrame columns, returning a tuple with</span>
<span class="sd"> the column name and the content as a Series.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> label : object</span>
<span class="sd"> The column names for the DataFrame being iterated over.</span>
<span class="sd"> content : Series</span>
<span class="sd"> The column entries belonging to each label, as a Series.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;species&#39;: [&#39;bear&#39;, &#39;bear&#39;, &#39;marsupial&#39;],</span>
<span class="sd"> ... &#39;population&#39;: [1864, 22000, 80000]},</span>
<span class="sd"> ... index=[&#39;panda&#39;, &#39;polar&#39;, &#39;koala&#39;],</span>
<span class="sd"> ... columns=[&#39;species&#39;, &#39;population&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> species population</span>
<span class="sd"> panda bear 1864</span>
<span class="sd"> polar bear 22000</span>
<span class="sd"> koala marsupial 80000</span>
<span class="sd"> &gt;&gt;&gt; for label, content in df.items():</span>
<span class="sd"> ... print(&#39;label:&#39;, label)</span>
<span class="sd"> ... print(&#39;content:&#39;, content.to_string())</span>
<span class="sd"> label: species</span>
<span class="sd"> content: panda bear</span>
<span class="sd"> polar bear</span>
<span class="sd"> koala marsupial</span>
<span class="sd"> label: population</span>
<span class="sd"> content: panda 1864</span>
<span class="sd"> polar 22000</span>
<span class="sd"> koala 80000</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="p">(</span>
<span class="p">(</span><span class="n">label</span> <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</span> <span class="k">else</span> <span class="n">label</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">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.iterrows"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.iterrows.html#pyspark.pandas.DataFrame.iterrows">[docs]</a> <span class="k">def</span> <span class="nf">iterrows</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterator</span><span class="p">[</span><span class="n">Tuple</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">]]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Iterate over DataFrame rows as (index, Series) pairs.</span>
<span class="sd"> Yields</span>
<span class="sd"> ------</span>
<span class="sd"> index : label or tuple of label</span>
<span class="sd"> The index of the row. A tuple for a `MultiIndex`.</span>
<span class="sd"> data : pandas.Series</span>
<span class="sd"> The data of the row as a Series.</span>
<span class="sd"> it : generator</span>
<span class="sd"> A generator that iterates over the rows of the frame.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> 1. Because ``iterrows`` returns a Series for each row,</span>
<span class="sd"> it does **not** preserve dtypes across the rows (dtypes are</span>
<span class="sd"> preserved across columns for DataFrames). For example,</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[1, 1.5]], columns=[&#39;int&#39;, &#39;float&#39;])</span>
<span class="sd"> &gt;&gt;&gt; row = next(df.iterrows())[1]</span>
<span class="sd"> &gt;&gt;&gt; row</span>
<span class="sd"> int 1.0</span>
<span class="sd"> float 1.5</span>
<span class="sd"> Name: 0, dtype: float64</span>
<span class="sd"> &gt;&gt;&gt; print(row[&#39;int&#39;].dtype)</span>
<span class="sd"> float64</span>
<span class="sd"> &gt;&gt;&gt; print(df[&#39;int&#39;].dtype)</span>
<span class="sd"> int64</span>
<span class="sd"> To preserve dtypes while iterating over the rows, it is better</span>
<span class="sd"> to use :meth:`itertuples` which returns namedtuples of the values</span>
<span class="sd"> and which is generally faster than ``iterrows``.</span>
<span class="sd"> 2. You should **never modify** something you are iterating over.</span>
<span class="sd"> This is not guaranteed to work in all cases. Depending on the</span>
<span class="sd"> data types, the iterator returns a copy and not a view, and writing</span>
<span class="sd"> to it will have no effect.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">columns</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span>
<span class="n">internal_index_columns</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">internal_data_columns</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="k">def</span> <span class="nf">extract_kv_from_spark_row</span><span class="p">(</span><span class="n">row</span><span class="p">:</span> <span class="n">Row</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Tuple</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Any</span><span class="p">]:</span>
<span class="n">k</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">row</span><span class="p">[</span><span class="n">internal_index_columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">internal_index_columns</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span>
<span class="k">else</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">row</span><span class="p">[</span><span class="n">c</span><span class="p">]</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">internal_index_columns</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">v</span> <span class="o">=</span> <span class="p">[</span><span class="n">row</span><span class="p">[</span><span class="n">c</span><span class="p">]</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">internal_data_columns</span><span class="p">]</span>
<span class="k">return</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="nb">map</span><span class="p">(</span>
<span class="n">extract_kv_from_spark_row</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">toLocalIterator</span><span class="p">()</span>
<span class="p">):</span>
<span class="n">s</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="n">columns</span><span class="p">,</span> <span class="n">name</span><span class="o">=</span><span class="n">k</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">k</span><span class="p">,</span> <span class="n">s</span></div>
<div class="viewcode-block" id="DataFrame.itertuples"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.itertuples.html#pyspark.pandas.DataFrame.itertuples">[docs]</a> <span class="k">def</span> <span class="nf">itertuples</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</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="s2">&quot;PandasOnSpark&quot;</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterator</span><span class="p">[</span><span class="n">Tuple</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Iterate over DataFrame rows as namedtuples.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> index : bool, default True</span>
<span class="sd"> If True, return the index as the first element of the tuple.</span>
<span class="sd"> name : str or None, default &quot;PandasOnSpark&quot;</span>
<span class="sd"> The name of the returned namedtuples or None to return regular</span>
<span class="sd"> tuples.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> iterator</span>
<span class="sd"> An object to iterate over namedtuples for each row in the</span>
<span class="sd"> DataFrame with the first field possibly being the index and</span>
<span class="sd"> following fields being the column values.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.iterrows : Iterate over DataFrame rows as (index, Series)</span>
<span class="sd"> pairs.</span>
<span class="sd"> DataFrame.items : Iterate over (column name, Series) pairs.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> The column names will be renamed to positional names if they are</span>
<span class="sd"> invalid Python identifiers, repeated, or start with an underscore.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;num_legs&#39;: [4, 2], &#39;num_wings&#39;: [0, 2]},</span>
<span class="sd"> ... index=[&#39;dog&#39;, &#39;hawk&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> num_legs num_wings</span>
<span class="sd"> dog 4 0</span>
<span class="sd"> hawk 2 2</span>
<span class="sd"> &gt;&gt;&gt; for row in df.itertuples():</span>
<span class="sd"> ... print(row)</span>
<span class="sd"> ...</span>
<span class="sd"> PandasOnSpark(Index=&#39;dog&#39;, num_legs=4, num_wings=0)</span>
<span class="sd"> PandasOnSpark(Index=&#39;hawk&#39;, num_legs=2, num_wings=2)</span>
<span class="sd"> By setting the `index` parameter to False we can remove the index</span>
<span class="sd"> as the first element of the tuple:</span>
<span class="sd"> &gt;&gt;&gt; for row in df.itertuples(index=False):</span>
<span class="sd"> ... print(row)</span>
<span class="sd"> ...</span>
<span class="sd"> PandasOnSpark(num_legs=4, num_wings=0)</span>
<span class="sd"> PandasOnSpark(num_legs=2, num_wings=2)</span>
<span class="sd"> With the `name` parameter set we set a custom name for the yielded</span>
<span class="sd"> namedtuples:</span>
<span class="sd"> &gt;&gt;&gt; for row in df.itertuples(name=&#39;Animal&#39;):</span>
<span class="sd"> ... print(row)</span>
<span class="sd"> ...</span>
<span class="sd"> Animal(Index=&#39;dog&#39;, num_legs=4, num_wings=0)</span>
<span class="sd"> Animal(Index=&#39;hawk&#39;, num_legs=2, num_wings=2)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">fields</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span>
<span class="k">if</span> <span class="n">index</span><span class="p">:</span>
<span class="n">fields</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="s2">&quot;Index&quot;</span><span class="p">)</span>
<span class="n">index_spark_column_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">data_spark_column_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="k">def</span> <span class="nf">extract_kv_from_spark_row</span><span class="p">(</span><span class="n">row</span><span class="p">:</span> <span class="n">Row</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Tuple</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Any</span><span class="p">]:</span>
<span class="n">k</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">row</span><span class="p">[</span><span class="n">index_spark_column_names</span><span class="p">[</span><span class="mi">0</span><span class="p">]]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">index_spark_column_names</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span>
<span class="k">else</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">row</span><span class="p">[</span><span class="n">c</span><span class="p">]</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">index_spark_column_names</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">v</span> <span class="o">=</span> <span class="p">[</span><span class="n">row</span><span class="p">[</span><span class="n">c</span><span class="p">]</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">data_spark_column_names</span><span class="p">]</span>
<span class="k">return</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</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="n">itertuple</span> <span class="o">=</span> <span class="n">namedtuple</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">fields</span><span class="p">,</span> <span class="n">rename</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> <span class="c1"># type: ignore[misc]</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="nb">map</span><span class="p">(</span>
<span class="n">extract_kv_from_spark_row</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">toLocalIterator</span><span class="p">(),</span>
<span class="p">):</span>
<span class="k">yield</span> <span class="n">itertuple</span><span class="o">.</span><span class="n">_make</span><span class="p">(([</span><span class="n">k</span><span class="p">]</span> <span class="k">if</span> <span class="n">index</span> <span class="k">else</span> <span class="p">[])</span> <span class="o">+</span> <span class="nb">list</span><span class="p">(</span><span class="n">v</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="nb">map</span><span class="p">(</span>
<span class="n">extract_kv_from_spark_row</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">toLocalIterator</span><span class="p">(),</span>
<span class="p">):</span>
<span class="k">yield</span> <span class="nb">tuple</span><span class="p">(([</span><span class="n">k</span><span class="p">]</span> <span class="k">if</span> <span class="n">index</span> <span class="k">else</span> <span class="p">[])</span> <span class="o">+</span> <span class="nb">list</span><span class="p">(</span><span class="n">v</span><span class="p">))</span></div>
<div class="viewcode-block" id="DataFrame.to_clipboard"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_clipboard.html#pyspark.pandas.DataFrame.to_clipboard">[docs]</a> <span class="k">def</span> <span class="nf">to_clipboard</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">excel</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span> <span class="n">sep</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">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</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;</span>
<span class="sd"> Copy object to the system clipboard.</span>
<span class="sd"> Write a text representation of object to the system clipboard.</span>
<span class="sd"> This can be pasted into Excel, for example.</span>
<span class="sd"> .. note:: This method should only be used if the resulting DataFrame is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> excel : bool, default True</span>
<span class="sd"> - True, use the provided separator, writing in a csv format for</span>
<span class="sd"> allowing easy pasting into excel.</span>
<span class="sd"> - False, write a string representation of the object to the</span>
<span class="sd"> clipboard.</span>
<span class="sd"> sep : str, default ``&#39;\\t&#39;``</span>
<span class="sd"> Field delimiter.</span>
<span class="sd"> **kwargs</span>
<span class="sd"> These parameters will be passed to DataFrame.to_csv.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> Requirements for your platform.</span>
<span class="sd"> - Linux : `xclip`, or `xsel` (with `gtk` or `PyQt4` modules)</span>
<span class="sd"> - Windows : none</span>
<span class="sd"> - OS X : none</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> read_clipboard : Read text from clipboard.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> Copy the contents of a DataFrame to the clipboard.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[1, 2, 3], [4, 5, 6]], columns=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;]) # doctest: +SKIP</span>
<span class="sd"> &gt;&gt;&gt; df.to_clipboard(sep=&#39;,&#39;) # doctest: +SKIP</span>
<span class="sd"> ... # Wrote the following to the system clipboard:</span>
<span class="sd"> ... # ,A,B,C</span>
<span class="sd"> ... # 0,1,2,3</span>
<span class="sd"> ... # 1,4,5,6</span>
<span class="sd"> We can omit the index by passing the keyword `index` and setting</span>
<span class="sd"> it to false.</span>
<span class="sd"> &gt;&gt;&gt; df.to_clipboard(sep=&#39;,&#39;, index=False) # doctest: +SKIP</span>
<span class="sd"> ... # Wrote the following to the system clipboard:</span>
<span class="sd"> ... # A,B,C</span>
<span class="sd"> ... # 1,2,3</span>
<span class="sd"> ... # 4,5,6</span>
<span class="sd"> This function also works for Series:</span>
<span class="sd"> &gt;&gt;&gt; df = ps.Series([1, 2, 3, 4, 5, 6, 7], name=&#39;x&#39;) # doctest: +SKIP</span>
<span class="sd"> &gt;&gt;&gt; df.to_clipboard(sep=&#39;,&#39;) # doctest: +SKIP</span>
<span class="sd"> ... # Wrote the following to the system clipboard:</span>
<span class="sd"> ... # 0, 1</span>
<span class="sd"> ... # 1, 2</span>
<span class="sd"> ... # 2, 3</span>
<span class="sd"> ... # 3, 4</span>
<span class="sd"> ... # 4, 5</span>
<span class="sd"> ... # 5, 6</span>
<span class="sd"> ... # 6, 7</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">args</span> <span class="o">=</span> <span class="nb">locals</span><span class="p">()</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span>
<span class="k">return</span> <span class="n">validate_arguments_and_invoke_function</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_clipboard</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_clipboard</span><span class="p">,</span> <span class="n">args</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_html"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_html.html#pyspark.pandas.DataFrame.to_html">[docs]</a> <span class="k">def</span> <span class="nf">to_html</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">buf</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">IO</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="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Sequence</span><span class="p">[</span><span class="n">Name</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">col_space</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">int</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">int</span><span class="p">]]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">header</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">na_rep</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;NaN&quot;</span><span class="p">,</span>
<span class="n">formatters</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span>
<span class="n">Union</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="nb">str</span><span class="p">]],</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="nb">str</span><span class="p">]]]</span>
<span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">float_format</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Callable</span><span class="p">[[</span><span class="nb">float</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="n">sparsify</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index_names</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">justify</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="n">max_rows</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">max_cols</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">show_dimensions</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">decimal</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;.&quot;</span><span class="p">,</span>
<span class="n">bold_rows</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">classes</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">list</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">escape</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">notebook</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">border</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">table_id</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="n">render_links</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Render a DataFrame as an HTML table.</span>
<span class="sd"> .. note:: This method should only be used if the resulting pandas object is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory. If the input</span>
<span class="sd"> is large, set max_rows parameter.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> buf : StringIO-like, optional</span>
<span class="sd"> Buffer to write to.</span>
<span class="sd"> columns : sequence, optional, default None</span>
<span class="sd"> The subset of columns to write. Writes all columns by default.</span>
<span class="sd"> col_space : int, optional</span>
<span class="sd"> The minimum width of each column.</span>
<span class="sd"> header : bool, optional</span>
<span class="sd"> Write out the column names. If a list of strings is given, it</span>
<span class="sd"> is assumed to be aliases for the column names</span>
<span class="sd"> index : bool, optional, default True</span>
<span class="sd"> Whether to print index (row) labels.</span>
<span class="sd"> na_rep : str, optional, default &#39;NaN&#39;</span>
<span class="sd"> String representation of NAN to use.</span>
<span class="sd"> formatters : list or dict of one-param. functions, optional</span>
<span class="sd"> Formatter functions to apply to columns&#39; elements by position or</span>
<span class="sd"> name.</span>
<span class="sd"> The result of each function must be a Unicode string.</span>
<span class="sd"> List must be of length equal to the number of columns.</span>
<span class="sd"> float_format : one-parameter function, optional, default None</span>
<span class="sd"> Formatter function to apply to columns&#39; elements if they are</span>
<span class="sd"> floats. The result of this function must be a Unicode string.</span>
<span class="sd"> sparsify : bool, optional, default True</span>
<span class="sd"> Set to False for a DataFrame with a hierarchical index to print</span>
<span class="sd"> every multiindex key at each row.</span>
<span class="sd"> index_names : bool, optional, default True</span>
<span class="sd"> Prints the names of the indexes.</span>
<span class="sd"> justify : str, default None</span>
<span class="sd"> How to justify the column labels. If None uses the option from</span>
<span class="sd"> the print configuration (controlled by set_option), &#39;right&#39; out</span>
<span class="sd"> of the box. Valid values are</span>
<span class="sd"> * left</span>
<span class="sd"> * right</span>
<span class="sd"> * center</span>
<span class="sd"> * justify</span>
<span class="sd"> * justify-all</span>
<span class="sd"> * start</span>
<span class="sd"> * end</span>
<span class="sd"> * inherit</span>
<span class="sd"> * match-parent</span>
<span class="sd"> * initial</span>
<span class="sd"> * unset.</span>
<span class="sd"> max_rows : int, optional</span>
<span class="sd"> Maximum number of rows to display in the console.</span>
<span class="sd"> max_cols : int, optional</span>
<span class="sd"> Maximum number of columns to display in the console.</span>
<span class="sd"> show_dimensions : bool, default False</span>
<span class="sd"> Display DataFrame dimensions (number of rows by number of columns).</span>
<span class="sd"> decimal : str, default &#39;.&#39;</span>
<span class="sd"> Character recognized as decimal separator, e.g. &#39;,&#39; in Europe.</span>
<span class="sd"> bold_rows : bool, default True</span>
<span class="sd"> Make the row labels bold in the output.</span>
<span class="sd"> classes : str or list or tuple, default None</span>
<span class="sd"> CSS class(es) to apply to the resulting html table.</span>
<span class="sd"> escape : bool, default True</span>
<span class="sd"> Convert the characters &lt;, &gt;, and &amp; to HTML-safe sequences.</span>
<span class="sd"> notebook : {True, False}, default False</span>
<span class="sd"> Whether the generated HTML is for IPython Notebook.</span>
<span class="sd"> border : int</span>
<span class="sd"> A ``border=border`` attribute is included in the opening</span>
<span class="sd"> `&lt;table&gt;` tag. By default ``pd.options.html.border``.</span>
<span class="sd"> table_id : str, optional</span>
<span class="sd"> A css id is included in the opening `&lt;table&gt;` tag if specified.</span>
<span class="sd"> render_links : bool, default False</span>
<span class="sd"> Convert URLs to HTML links (only works with pandas 0.24+).</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> str (or Unicode, depending on data and options)</span>
<span class="sd"> String representation of the dataframe.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> to_string : Convert DataFrame to a string.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Make sure locals() call is at the top of the function so we don&#39;t capture local variables.</span>
<span class="n">args</span> <span class="o">=</span> <span class="nb">locals</span><span class="p">()</span>
<span class="k">if</span> <span class="n">max_rows</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">max_rows</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span>
<span class="k">return</span> <span class="n">validate_arguments_and_invoke_function</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_html</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_html</span><span class="p">,</span> <span class="n">args</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_string"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_string.html#pyspark.pandas.DataFrame.to_string">[docs]</a> <span class="k">def</span> <span class="nf">to_string</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">buf</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">IO</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="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Sequence</span><span class="p">[</span><span class="n">Name</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">col_space</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">int</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">int</span><span class="p">]]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">header</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">na_rep</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;NaN&quot;</span><span class="p">,</span>
<span class="n">formatters</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span>
<span class="n">Union</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="nb">str</span><span class="p">]],</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="nb">str</span><span class="p">]]]</span>
<span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">float_format</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Callable</span><span class="p">[[</span><span class="nb">float</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="n">sparsify</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index_names</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">justify</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="n">max_rows</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">max_cols</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">show_dimensions</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">decimal</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;.&quot;</span><span class="p">,</span>
<span class="n">line_width</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Render a DataFrame to a console-friendly tabular output.</span>
<span class="sd"> .. note:: This method should only be used if the resulting pandas object is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory. If the input</span>
<span class="sd"> is large, set max_rows parameter.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> buf : StringIO-like, optional</span>
<span class="sd"> Buffer to write to.</span>
<span class="sd"> columns : sequence, optional, default None</span>
<span class="sd"> The subset of columns to write. Writes all columns by default.</span>
<span class="sd"> col_space : int, optional</span>
<span class="sd"> The minimum width of each column.</span>
<span class="sd"> header : bool, optional</span>
<span class="sd"> Write out the column names. If a list of strings is given, it</span>
<span class="sd"> is assumed to be aliases for the column names</span>
<span class="sd"> index : bool, optional, default True</span>
<span class="sd"> Whether to print index (row) labels.</span>
<span class="sd"> na_rep : str, optional, default &#39;NaN&#39;</span>
<span class="sd"> String representation of NAN to use.</span>
<span class="sd"> formatters : list or dict of one-param. functions, optional</span>
<span class="sd"> Formatter functions to apply to columns&#39; elements by position or</span>
<span class="sd"> name.</span>
<span class="sd"> The result of each function must be a Unicode string.</span>
<span class="sd"> List must be of length equal to the number of columns.</span>
<span class="sd"> float_format : one-parameter function, optional, default None</span>
<span class="sd"> Formatter function to apply to columns&#39; elements if they are</span>
<span class="sd"> floats. The result of this function must be a Unicode string.</span>
<span class="sd"> sparsify : bool, optional, default True</span>
<span class="sd"> Set to False for a DataFrame with a hierarchical index to print</span>
<span class="sd"> every multiindex key at each row.</span>
<span class="sd"> index_names : bool, optional, default True</span>
<span class="sd"> Prints the names of the indexes.</span>
<span class="sd"> justify : str, default None</span>
<span class="sd"> How to justify the column labels. If None uses the option from</span>
<span class="sd"> the print configuration (controlled by set_option), &#39;right&#39; out</span>
<span class="sd"> of the box. Valid values are</span>
<span class="sd"> * left</span>
<span class="sd"> * right</span>
<span class="sd"> * center</span>
<span class="sd"> * justify</span>
<span class="sd"> * justify-all</span>
<span class="sd"> * start</span>
<span class="sd"> * end</span>
<span class="sd"> * inherit</span>
<span class="sd"> * match-parent</span>
<span class="sd"> * initial</span>
<span class="sd"> * unset.</span>
<span class="sd"> max_rows : int, optional</span>
<span class="sd"> Maximum number of rows to display in the console.</span>
<span class="sd"> max_cols : int, optional</span>
<span class="sd"> Maximum number of columns to display in the console.</span>
<span class="sd"> show_dimensions : bool, default False</span>
<span class="sd"> Display DataFrame dimensions (number of rows by number of columns).</span>
<span class="sd"> decimal : str, default &#39;.&#39;</span>
<span class="sd"> Character recognized as decimal separator, e.g. &#39;,&#39; in Europe.</span>
<span class="sd"> line_width : int, optional</span>
<span class="sd"> Width to wrap a line in characters.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> str (or Unicode, depending on data and options)</span>
<span class="sd"> String representation of the dataframe.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> to_html : Convert DataFrame to HTML.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;col1&#39;: [1, 2, 3], &#39;col2&#39;: [4, 5, 6]}, columns=[&#39;col1&#39;, &#39;col2&#39;])</span>
<span class="sd"> &gt;&gt;&gt; print(df.to_string())</span>
<span class="sd"> col1 col2</span>
<span class="sd"> 0 1 4</span>
<span class="sd"> 1 2 5</span>
<span class="sd"> 2 3 6</span>
<span class="sd"> &gt;&gt;&gt; print(df.to_string(max_rows=2))</span>
<span class="sd"> col1 col2</span>
<span class="sd"> 0 1 4</span>
<span class="sd"> 1 2 5</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Make sure locals() call is at the top of the function so we don&#39;t capture local variables.</span>
<span class="n">args</span> <span class="o">=</span> <span class="nb">locals</span><span class="p">()</span>
<span class="k">if</span> <span class="n">max_rows</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">max_rows</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span>
<span class="k">return</span> <span class="n">validate_arguments_and_invoke_function</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_string</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_string</span><span class="p">,</span> <span class="n">args</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_dict"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_dict.html#pyspark.pandas.DataFrame.to_dict">[docs]</a> <span class="k">def</span> <span class="nf">to_dict</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">orient</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;dict&quot;</span><span class="p">,</span> <span class="n">into</span><span class="p">:</span> <span class="n">Type</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Union</span><span class="p">[</span><span class="n">List</span><span class="p">,</span> <span class="n">Mapping</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Convert the DataFrame to a dictionary.</span>
<span class="sd"> The type of the key-value pairs can be customized with the parameters</span>
<span class="sd"> (see below).</span>
<span class="sd"> .. note:: This method should only be used if the resulting pandas DataFrame is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> orient : str {&#39;dict&#39;, &#39;list&#39;, &#39;series&#39;, &#39;split&#39;, &#39;records&#39;, &#39;index&#39;}</span>
<span class="sd"> Determines the type of the values of the dictionary.</span>
<span class="sd"> - &#39;dict&#39; (default) : dict like {column -&gt; {index -&gt; value}}</span>
<span class="sd"> - &#39;list&#39; : dict like {column -&gt; [values]}</span>
<span class="sd"> - &#39;series&#39; : dict like {column -&gt; Series(values)}</span>
<span class="sd"> - &#39;split&#39; : dict like</span>
<span class="sd"> {&#39;index&#39; -&gt; [index], &#39;columns&#39; -&gt; [columns], &#39;data&#39; -&gt; [values]}</span>
<span class="sd"> - &#39;records&#39; : list like</span>
<span class="sd"> [{column -&gt; value}, ... , {column -&gt; value}]</span>
<span class="sd"> - &#39;index&#39; : dict like {index -&gt; {column -&gt; value}}</span>
<span class="sd"> Abbreviations are allowed. `s` indicates `series` and `sp`</span>
<span class="sd"> indicates `split`.</span>
<span class="sd"> into : class, default dict</span>
<span class="sd"> The collections.abc.Mapping subclass used for all Mappings</span>
<span class="sd"> in the return value. Can be the actual class or an empty</span>
<span class="sd"> instance of the mapping type you want. If you want a</span>
<span class="sd"> collections.defaultdict, you must pass it initialized.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> dict, list or collections.abc.Mapping</span>
<span class="sd"> Return a collections.abc.Mapping object representing the DataFrame.</span>
<span class="sd"> The resulting transformation depends on the `orient` parameter.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;col1&#39;: [1, 2],</span>
<span class="sd"> ... &#39;col2&#39;: [0.5, 0.75]},</span>
<span class="sd"> ... index=[&#39;row1&#39;, &#39;row2&#39;],</span>
<span class="sd"> ... columns=[&#39;col1&#39;, &#39;col2&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> col1 col2</span>
<span class="sd"> row1 1 0.50</span>
<span class="sd"> row2 2 0.75</span>
<span class="sd"> &gt;&gt;&gt; df_dict = df.to_dict()</span>
<span class="sd"> &gt;&gt;&gt; sorted([(key, sorted(values.items())) for key, values in df_dict.items()])</span>
<span class="sd"> [(&#39;col1&#39;, [(&#39;row1&#39;, 1), (&#39;row2&#39;, 2)]), (&#39;col2&#39;, [(&#39;row1&#39;, 0.5), (&#39;row2&#39;, 0.75)])]</span>
<span class="sd"> You can specify the return orientation.</span>
<span class="sd"> &gt;&gt;&gt; df_dict = df.to_dict(&#39;series&#39;)</span>
<span class="sd"> &gt;&gt;&gt; sorted(df_dict.items())</span>
<span class="sd"> [(&#39;col1&#39;, row1 1</span>
<span class="sd"> row2 2</span>
<span class="sd"> Name: col1, dtype: int64), (&#39;col2&#39;, row1 0.50</span>
<span class="sd"> row2 0.75</span>
<span class="sd"> Name: col2, dtype: float64)]</span>
<span class="sd"> &gt;&gt;&gt; df_dict = df.to_dict(&#39;split&#39;)</span>
<span class="sd"> &gt;&gt;&gt; sorted(df_dict.items()) # doctest: +ELLIPSIS</span>
<span class="sd"> [(&#39;columns&#39;, [&#39;col1&#39;, &#39;col2&#39;]), (&#39;data&#39;, [[1..., 0.75]]), (&#39;index&#39;, [&#39;row1&#39;, &#39;row2&#39;])]</span>
<span class="sd"> &gt;&gt;&gt; df_dict = df.to_dict(&#39;records&#39;)</span>
<span class="sd"> &gt;&gt;&gt; [sorted(values.items()) for values in df_dict] # doctest: +ELLIPSIS</span>
<span class="sd"> [[(&#39;col1&#39;, 1...), (&#39;col2&#39;, 0.5)], [(&#39;col1&#39;, 2...), (&#39;col2&#39;, 0.75)]]</span>
<span class="sd"> &gt;&gt;&gt; df_dict = df.to_dict(&#39;index&#39;)</span>
<span class="sd"> &gt;&gt;&gt; sorted([(key, sorted(values.items())) for key, values in df_dict.items()])</span>
<span class="sd"> [(&#39;row1&#39;, [(&#39;col1&#39;, 1), (&#39;col2&#39;, 0.5)]), (&#39;row2&#39;, [(&#39;col1&#39;, 2), (&#39;col2&#39;, 0.75)])]</span>
<span class="sd"> You can also specify the mapping type.</span>
<span class="sd"> &gt;&gt;&gt; from collections import OrderedDict, defaultdict</span>
<span class="sd"> &gt;&gt;&gt; df.to_dict(into=OrderedDict) # doctest: +ELLIPSIS</span>
<span class="sd"> OrderedDict(...)</span>
<span class="sd"> If you want a `defaultdict`, you need to initialize it:</span>
<span class="sd"> &gt;&gt;&gt; dd = defaultdict(list)</span>
<span class="sd"> &gt;&gt;&gt; df.to_dict(&#39;records&#39;, into=dd) # doctest: +ELLIPSIS</span>
<span class="sd"> [defaultdict(&lt;class &#39;list&#39;&gt;, {&#39;col..., &#39;col...}), \</span>
<span class="sd">defaultdict(&lt;class &#39;list&#39;&gt;, {&#39;col..., &#39;col...})]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Make sure locals() call is at the top of the function so we don&#39;t capture local variables.</span>
<span class="n">args</span> <span class="o">=</span> <span class="nb">locals</span><span class="p">()</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span>
<span class="k">return</span> <span class="n">validate_arguments_and_invoke_function</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_dict</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_dict</span><span class="p">,</span> <span class="n">args</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_latex"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_latex.html#pyspark.pandas.DataFrame.to_latex">[docs]</a> <span class="k">def</span> <span class="nf">to_latex</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">buf</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">IO</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="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">header</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">na_rep</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;NaN&quot;</span><span class="p">,</span>
<span class="n">formatters</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span>
<span class="n">Union</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="nb">str</span><span class="p">]],</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="nb">str</span><span class="p">]]]</span>
<span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">float_format</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Callable</span><span class="p">[[</span><span class="nb">float</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="n">sparsify</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index_names</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">bold_rows</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">column_format</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="n">longtable</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">escape</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">encoding</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="n">decimal</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;.&quot;</span><span class="p">,</span>
<span class="n">multicolumn</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">multicolumn_format</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="n">multirow</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]:</span>
<span class="w"> </span><span class="sa">r</span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Render an object to a LaTeX tabular environment table.</span>
<span class="sd"> Render an object to a tabular environment table. You can splice this into a LaTeX</span>
<span class="sd"> document. Requires usepackage{booktabs}.</span>
<span class="sd"> .. note:: This method should only be used if the resulting pandas object is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory. If the input</span>
<span class="sd"> is large, consider alternative formats.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> buf : file descriptor or None</span>
<span class="sd"> Buffer to write to. If None, the output is returned as a string.</span>
<span class="sd"> columns : list of label, optional</span>
<span class="sd"> The subset of columns to write. Writes all columns by default.</span>
<span class="sd"> header : bool or list of str, default True</span>
<span class="sd"> Write out the column names. If a list of strings is given, it is assumed to be aliases</span>
<span class="sd"> for the column names.</span>
<span class="sd"> index : bool, default True</span>
<span class="sd"> Write row names (index).</span>
<span class="sd"> na_rep : str, default ‘NaN’</span>
<span class="sd"> Missing data representation.</span>
<span class="sd"> formatters : list of functions or dict of {str: function}, optional</span>
<span class="sd"> Formatter functions to apply to columns’ elements by position or name. The result of</span>
<span class="sd"> each function must be a Unicode string. List must be of length equal to the number of</span>
<span class="sd"> columns.</span>
<span class="sd"> float_format : str, optional</span>
<span class="sd"> Format string for floating point numbers.</span>
<span class="sd"> sparsify : bool, optional</span>
<span class="sd"> Set to False for a DataFrame with a hierarchical index to print every multiindex key at</span>
<span class="sd"> each row. By default the value will be read from the config module.</span>
<span class="sd"> index_names : bool, default True</span>
<span class="sd"> Prints the names of the indexes.</span>
<span class="sd"> bold_rows : bool, default False</span>
<span class="sd"> Make the row labels bold in the output.</span>
<span class="sd"> column_format : str, optional</span>
<span class="sd"> The columns format as specified in LaTeX table format e.g. ‘rcl’ for 3 columns. By</span>
<span class="sd"> default, ‘l’ will be used for all columns except columns of numbers, which default</span>
<span class="sd"> to ‘r’.</span>
<span class="sd"> longtable : bool, optional</span>
<span class="sd"> By default the value will be read from the pandas config module. Use a longtable</span>
<span class="sd"> environment instead of tabular. Requires adding a usepackage{longtable} to your LaTeX</span>
<span class="sd"> preamble.</span>
<span class="sd"> escape : bool, optional</span>
<span class="sd"> By default the value will be read from the pandas config module. When set to False</span>
<span class="sd"> prevents from escaping latex special characters in column names.</span>
<span class="sd"> encoding : str, optional</span>
<span class="sd"> A string representing the encoding to use in the output file, defaults to ‘ascii’ on</span>
<span class="sd"> Python 2 and ‘utf-8’ on Python 3.</span>
<span class="sd"> decimal : str, default ‘.’</span>
<span class="sd"> Character recognized as decimal separator, e.g. ‘,’ in Europe.</span>
<span class="sd"> multicolumn : bool, default True</span>
<span class="sd"> Use multicolumn to enhance MultiIndex columns. The default will be read from the config</span>
<span class="sd"> module.</span>
<span class="sd"> multicolumn_format : str, default ‘l’</span>
<span class="sd"> The alignment for multicolumns, similar to column_format The default will be read from</span>
<span class="sd"> the config module.</span>
<span class="sd"> multirow : bool, default False</span>
<span class="sd"> Use multirow to enhance MultiIndex rows. Requires adding a usepackage{multirow} to your</span>
<span class="sd"> LaTeX preamble. Will print centered labels (instead of top-aligned) across the contained</span>
<span class="sd"> rows, separating groups via clines. The default will be read from the pandas config</span>
<span class="sd"> module.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> str or None</span>
<span class="sd"> If buf is None, returns the resulting LateX format as a string. Otherwise returns None.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.to_string : Render a DataFrame to a console-friendly</span>
<span class="sd"> tabular output.</span>
<span class="sd"> DataFrame.to_html : Render a DataFrame as an HTML table.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;name&#39;: [&#39;Raphael&#39;, &#39;Donatello&#39;],</span>
<span class="sd"> ... &#39;mask&#39;: [&#39;red&#39;, &#39;purple&#39;],</span>
<span class="sd"> ... &#39;weapon&#39;: [&#39;sai&#39;, &#39;bo staff&#39;]},</span>
<span class="sd"> ... columns=[&#39;name&#39;, &#39;mask&#39;, &#39;weapon&#39;])</span>
<span class="sd"> &gt;&gt;&gt; print(df.to_latex(index=False)) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> \begin{tabular}{lll}</span>
<span class="sd"> \toprule</span>
<span class="sd"> name &amp; mask &amp; weapon \\</span>
<span class="sd"> \midrule</span>
<span class="sd"> Raphael &amp; red &amp; sai \\</span>
<span class="sd"> Donatello &amp; purple &amp; bo staff \\</span>
<span class="sd"> \bottomrule</span>
<span class="sd"> \end{tabular}</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">args</span> <span class="o">=</span> <span class="nb">locals</span><span class="p">()</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span>
<span class="k">return</span> <span class="n">validate_arguments_and_invoke_function</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_latex</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_latex</span><span class="p">,</span> <span class="n">args</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_feather"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_feather.html#pyspark.pandas.DataFrame.to_feather">[docs]</a> <span class="k">def</span> <span class="nf">to_feather</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">path</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">IO</span><span class="p">[</span><span class="nb">str</span><span class="p">]],</span>
<span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</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;</span>
<span class="sd"> Write a DataFrame to the binary Feather format.</span>
<span class="sd"> .. note:: This method should only be used if the resulting DataFrame is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> .. versionadded:: 4.0.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> path : str, path object, file-like object</span>
<span class="sd"> String, path object (implementing ``os.PathLike[str]``), or file-like</span>
<span class="sd"> object implementing a binary ``write()`` function.</span>
<span class="sd"> **kwargs :</span>
<span class="sd"> Additional keywords passed to :func:`pyarrow.feather.write_feather`.</span>
<span class="sd"> This includes the `compression`, `compression_level`, `chunksize`</span>
<span class="sd"> and `version` keywords.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[1, 2, 3], [4, 5, 6]])</span>
<span class="sd"> &gt;&gt;&gt; df.to_feather(&quot;file.feather&quot;) # doctest: +SKIP</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Make sure locals() call is at the top of the function so we don&#39;t capture local variables.</span>
<span class="n">args</span> <span class="o">=</span> <span class="nb">locals</span><span class="p">()</span>
<span class="k">return</span> <span class="n">validate_arguments_and_invoke_function</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_feather</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_feather</span><span class="p">,</span> <span class="n">args</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_stata"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_stata.html#pyspark.pandas.DataFrame.to_stata">[docs]</a> <span class="k">def</span> <span class="nf">to_stata</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">path</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">IO</span><span class="p">[</span><span class="nb">str</span><span class="p">]],</span>
<span class="o">*</span><span class="p">,</span>
<span class="n">convert_dates</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Dict</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">write_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">byteorder</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="n">time_stamp</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">datetime</span><span class="o">.</span><span class="n">datetime</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">data_label</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="n">variable_labels</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Dict</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">version</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="mi">114</span><span class="p">,</span>
<span class="n">convert_strl</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Sequence</span><span class="p">[</span><span class="n">Name</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">compression</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;infer&quot;</span><span class="p">,</span>
<span class="n">storage_options</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="n">value_labels</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Dict</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</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;</span>
<span class="sd"> Export DataFrame object to Stata dta format.</span>
<span class="sd"> .. note:: This method should only be used if the resulting DataFrame is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> .. versionadded:: 4.0.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> path : str, path object, or buffer</span>
<span class="sd"> String, path object (implementing ``os.PathLike[str]``), or file-like</span>
<span class="sd"> object implementing a binary ``write()`` function.</span>
<span class="sd"> convert_dates : dict</span>
<span class="sd"> Dictionary mapping columns containing datetime types to stata</span>
<span class="sd"> internal format to use when writing the dates. Options are &#39;tc&#39;,</span>
<span class="sd"> &#39;td&#39;, &#39;tm&#39;, &#39;tw&#39;, &#39;th&#39;, &#39;tq&#39;, &#39;ty&#39;. Column can be either an integer</span>
<span class="sd"> or a name. Datetime columns that do not have a conversion type</span>
<span class="sd"> specified will be converted to &#39;tc&#39;. Raises NotImplementedError if</span>
<span class="sd"> a datetime column has timezone information.</span>
<span class="sd"> write_index : bool</span>
<span class="sd"> Write the index to Stata dataset.</span>
<span class="sd"> byteorder : str</span>
<span class="sd"> Can be &quot;&gt;&quot;, &quot;&lt;&quot;, &quot;little&quot;, or &quot;big&quot;. default is `sys.byteorder`.</span>
<span class="sd"> time_stamp : datetime</span>
<span class="sd"> A datetime to use as file creation date. Default is the current</span>
<span class="sd"> time.</span>
<span class="sd"> data_label : str, optional</span>
<span class="sd"> A label for the data set. Must be 80 characters or smaller.</span>
<span class="sd"> variable_labels : dict</span>
<span class="sd"> Dictionary containing columns as keys and variable labels as</span>
<span class="sd"> values. Each label must be 80 characters or smaller.</span>
<span class="sd"> version : {{114, 117, 118, 119, None}}, default 114</span>
<span class="sd"> Version to use in the output dta file. Set to None to let pandas</span>
<span class="sd"> decide between 118 or 119 formats depending on the number of</span>
<span class="sd"> columns in the frame. Version 114 can be read by Stata 10 and</span>
<span class="sd"> later. Version 117 can be read by Stata 13 or later. Version 118</span>
<span class="sd"> is supported in Stata 14 and later. Version 119 is supported in</span>
<span class="sd"> Stata 15 and later. Version 114 limits string variables to 244</span>
<span class="sd"> characters or fewer while versions 117 and later allow strings</span>
<span class="sd"> with lengths up to 2,000,000 characters. Versions 118 and 119</span>
<span class="sd"> support Unicode characters, and version 119 supports more than</span>
<span class="sd"> 32,767 variables.</span>
<span class="sd"> convert_strl : list, optional</span>
<span class="sd"> List of column names to convert to string columns to Stata StrL</span>
<span class="sd"> format. Only available if version is 117. Storing strings in the</span>
<span class="sd"> StrL format can produce smaller dta files if strings have more than</span>
<span class="sd"> 8 characters and values are repeated.</span>
<span class="sd"> value_labels : dict of dicts</span>
<span class="sd"> Dictionary containing columns as keys and dictionaries of column value</span>
<span class="sd"> to labels as values. Labels for a single variable must be 32,000</span>
<span class="sd"> characters or smaller.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;animal&#39;: [&#39;falcon&#39;, &#39;parrot&#39;, &#39;falcon&#39;, &#39;parrot&#39;],</span>
<span class="sd"> ... &#39;speed&#39;: [350, 18, 361, 15]})</span>
<span class="sd"> &gt;&gt;&gt; df.to_stata(&#39;animals.dta&#39;) # doctest: +SKIP</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Make sure locals() call is at the top of the function so we don&#39;t capture local variables.</span>
<span class="n">args</span> <span class="o">=</span> <span class="nb">locals</span><span class="p">()</span>
<span class="k">return</span> <span class="n">validate_arguments_and_invoke_function</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_stata</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_stata</span><span class="p">,</span> <span class="n">args</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.transpose"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.transpose.html#pyspark.pandas.DataFrame.transpose">[docs]</a> <span class="k">def</span> <span class="nf">transpose</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Transpose index and columns.</span>
<span class="sd"> Reflect the DataFrame over its main diagonal by writing rows as columns</span>
<span class="sd"> and vice-versa. The property :attr:`.T` is an accessor to the method</span>
<span class="sd"> :meth:`transpose`.</span>
<span class="sd"> .. note:: This method is based on an expensive operation due to the nature</span>
<span class="sd"> of big data. Internally it needs to generate each row for each value, and</span>
<span class="sd"> then group twice - it is a huge operation. To prevent misuse, this method</span>
<span class="sd"> has the &#39;compute.max_rows&#39; default limit of input length and raises a ValueError.</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.pandas.config import option_context</span>
<span class="sd"> &gt;&gt;&gt; with option_context(&#39;compute.max_rows&#39;, 1000): # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> ... ps.DataFrame({&#39;a&#39;: range(1001)}).transpose()</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: Current DataFrame&#39;s length exceeds the given limit of 1000 rows.</span>
<span class="sd"> Please set &#39;compute.max_rows&#39; by using &#39;pyspark.pandas.config.set_option&#39;</span>
<span class="sd"> to retrieve more than 1000 rows. Note that, before changing the</span>
<span class="sd"> &#39;compute.max_rows&#39;, this operation is considerably expensive.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> The transposed DataFrame.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> Transposing a DataFrame with mixed dtypes will result in a homogeneous</span>
<span class="sd"> DataFrame with the coerced dtype. For instance, if int and float have</span>
<span class="sd"> to be placed in same column, it becomes float. If type coercion is not</span>
<span class="sd"> possible, it fails.</span>
<span class="sd"> Also, note that the values in index should be unique because they become</span>
<span class="sd"> unique column names.</span>
<span class="sd"> In addition, if Spark 2.3 is used, the types should always be exactly same.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> **Square DataFrame with homogeneous dtype**</span>
<span class="sd"> &gt;&gt;&gt; d1 = {&#39;col1&#39;: [1, 2], &#39;col2&#39;: [3, 4]}</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame(data=d1, columns=[&#39;col1&#39;, &#39;col2&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df1</span>
<span class="sd"> col1 col2</span>
<span class="sd"> 0 1 3</span>
<span class="sd"> 1 2 4</span>
<span class="sd"> &gt;&gt;&gt; df1_transposed = df1.T.sort_index()</span>
<span class="sd"> &gt;&gt;&gt; df1_transposed</span>
<span class="sd"> 0 1</span>
<span class="sd"> col1 1 2</span>
<span class="sd"> col2 3 4</span>
<span class="sd"> When the dtype is homogeneous in the original DataFrame, we get a</span>
<span class="sd"> transposed DataFrame with the same dtype:</span>
<span class="sd"> &gt;&gt;&gt; df1.dtypes</span>
<span class="sd"> col1 int64</span>
<span class="sd"> col2 int64</span>
<span class="sd"> dtype: object</span>
<span class="sd"> &gt;&gt;&gt; df1_transposed.dtypes</span>
<span class="sd"> 0 int64</span>
<span class="sd"> 1 int64</span>
<span class="sd"> dtype: object</span>
<span class="sd"> **Non-square DataFrame with mixed dtypes**</span>
<span class="sd"> &gt;&gt;&gt; d2 = {&#39;score&#39;: [9.5, 8],</span>
<span class="sd"> ... &#39;kids&#39;: [0, 0],</span>
<span class="sd"> ... &#39;age&#39;: [12, 22]}</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame(data=d2, columns=[&#39;score&#39;, &#39;kids&#39;, &#39;age&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df2</span>
<span class="sd"> score kids age</span>
<span class="sd"> 0 9.5 0 12</span>
<span class="sd"> 1 8.0 0 22</span>
<span class="sd"> &gt;&gt;&gt; df2_transposed = df2.T.sort_index()</span>
<span class="sd"> &gt;&gt;&gt; df2_transposed</span>
<span class="sd"> 0 1</span>
<span class="sd"> age 12.0 22.0</span>
<span class="sd"> kids 0.0 0.0</span>
<span class="sd"> score 9.5 8.0</span>
<span class="sd"> When the DataFrame has mixed dtypes, we get a transposed DataFrame with</span>
<span class="sd"> the coerced dtype:</span>
<span class="sd"> &gt;&gt;&gt; df2.dtypes</span>
<span class="sd"> score float64</span>
<span class="sd"> kids int64</span>
<span class="sd"> age int64</span>
<span class="sd"> dtype: object</span>
<span class="sd"> &gt;&gt;&gt; df2_transposed.dtypes</span>
<span class="sd"> 0 float64</span>
<span class="sd"> 1 float64</span>
<span class="sd"> dtype: object</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">max_compute_count</span> <span class="o">=</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.max_rows&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">max_compute_count</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">max_compute_count</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span> <span class="o">&gt;</span> <span class="n">max_compute_count</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Current DataFrame&#39;s length exceeds the given limit of </span><span class="si">{0}</span><span class="s2"> rows. &quot;</span>
<span class="s2">&quot;Please set &#39;compute.max_rows&#39; by using &#39;pyspark.pandas.config.set_option&#39; &quot;</span>
<span class="s2">&quot;to retrieve more than </span><span class="si">{0}</span><span class="s2"> rows. Note that, before changing the &quot;</span>
<span class="s2">&quot;&#39;compute.max_rows&#39;, this operation is considerably expensive.&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">max_compute_count</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">pdf</span><span class="o">.</span><span class="n">transpose</span><span class="p">())</span>
<span class="c1"># Explode the data to be pairs.</span>
<span class="c1">#</span>
<span class="c1"># For instance, if the current input DataFrame is as below:</span>
<span class="c1">#</span>
<span class="c1"># +------+------+------+------+------+</span>
<span class="c1"># |index1|index2|(a,x1)|(a,x2)|(b,x3)|</span>
<span class="c1"># +------+------+------+------+------+</span>
<span class="c1"># | y1| z1| 1| 0| 0|</span>
<span class="c1"># | y2| z2| 0| 50| 0|</span>
<span class="c1"># | y3| z3| 3| 2| 1|</span>
<span class="c1"># +------+------+------+------+------+</span>
<span class="c1">#</span>
<span class="c1"># Output of `exploded_df` becomes as below:</span>
<span class="c1">#</span>
<span class="c1"># +-----------------+-----------------+-----------------+-----+</span>
<span class="c1"># | index|__index_level_0__|__index_level_1__|value|</span>
<span class="c1"># +-----------------+-----------------+-----------------+-----+</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y1&quot;,&quot;z1&quot;]}| a| x1| 1|</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y1&quot;,&quot;z1&quot;]}| a| x2| 0|</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y1&quot;,&quot;z1&quot;]}| b| x3| 0|</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y2&quot;,&quot;z2&quot;]}| a| x1| 0|</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y2&quot;,&quot;z2&quot;]}| a| x2| 50|</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y2&quot;,&quot;z2&quot;]}| b| x3| 0|</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y3&quot;,&quot;z3&quot;]}| a| x1| 3|</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y3&quot;,&quot;z3&quot;]}| a| x2| 2|</span>
<span class="c1"># |{&quot;a&quot;:[&quot;y3&quot;,&quot;z3&quot;]}| b| x3| 1|</span>
<span class="c1"># +-----------------+-----------------+-----------------+-----+</span>
<span class="n">pairs</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">col</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="n">i</span><span class="p">))</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">col</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="p">],</span>
<span class="o">*</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="s2">&quot;value&quot;</span><span class="p">)],</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">]</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">exploded_df</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="s2">&quot;pairs&quot;</span><span class="p">,</span> <span class="n">pairs</span><span class="p">)</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">to_json</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span><span class="o">*</span><span class="p">[</span><span class="n">scol</span> <span class="k">for</span> <span class="n">scol</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">])</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="s2">&quot;a&quot;</span><span class="p">)</span>
<span class="p">)</span>
<span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="s2">&quot;index&quot;</span><span class="p">),</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;pairs.*&quot;</span><span class="p">),</span>
<span class="p">]</span>
<span class="p">)</span>
<span class="c1"># After that, executes pivot with key and its index column.</span>
<span class="c1"># Note that index column should contain unique values since column names</span>
<span class="c1"># should be unique.</span>
<span class="n">internal_index_columns</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="n">i</span><span class="p">)</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">pivoted_df</span> <span class="o">=</span> <span class="n">exploded_df</span><span class="o">.</span><span class="n">groupBy</span><span class="p">(</span><span class="n">internal_index_columns</span><span class="p">)</span><span class="o">.</span><span class="n">pivot</span><span class="p">(</span><span class="s2">&quot;index&quot;</span><span class="p">)</span>
<span class="n">transposed_df</span> <span class="o">=</span> <span class="n">pivoted_df</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">first</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;value&quot;</span><span class="p">)))</span>
<span class="n">new_data_columns</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span>
<span class="nb">filter</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">internal_index_columns</span><span class="p">,</span> <span class="n">transposed_df</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="kc">None</span> <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span> <span class="ow">and</span> <span class="n">label</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="n">label</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="p">(</span><span class="nb">tuple</span><span class="p">(</span><span class="n">json</span><span class="o">.</span><span class="n">loads</span><span class="p">(</span><span class="n">col</span><span class="p">)[</span><span class="s2">&quot;a&quot;</span><span class="p">])</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">new_data_columns</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">transposed_df</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">transposed_df</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">internal_index_columns</span><span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">transposed_df</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">new_data_columns</span><span class="p">],</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</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">internal</span><span class="p">)</span></div>
<span class="n">T</span> <span class="o">=</span> <span class="nb">property</span><span class="p">(</span><span class="n">transpose</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.apply"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.apply.html#pyspark.pandas.DataFrame.apply">[docs]</a> <span class="k">def</span> <span class="nf">apply</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">:</span> <span class="n">Callable</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="n">args</span><span class="p">:</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">]</span> <span class="o">=</span> <span class="p">(),</span> <span class="o">**</span><span class="n">kwds</span><span class="p">:</span> <span class="n">Any</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Union</span><span class="p">[</span><span class="s2">&quot;Series&quot;</span><span class="p">,</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">,</span> <span class="s2">&quot;Index&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Apply a function along an axis of the DataFrame.</span>
<span class="sd"> Objects passed to the function are Series objects whose index is</span>
<span class="sd"> either the DataFrame&#39;s index (``axis=0``) or the DataFrame&#39;s columns</span>
<span class="sd"> (``axis=1``).</span>
<span class="sd"> See also `Transform and apply a function</span>
<span class="sd"> &lt;https://spark.apache.org/docs/latest/api/python/user_guide/pandas_on_spark/transform_apply.html&gt;`_.</span>
<span class="sd"> .. note:: when `axis` is 0 or &#39;index&#39;, the `func` is unable to access</span>
<span class="sd"> to the whole input series. pandas-on-Spark internally splits the input series into</span>
<span class="sd"> multiple batches and calls `func` with each batch multiple times. Therefore, operations</span>
<span class="sd"> such as global aggregations are impossible. See the example below.</span>
<span class="sd"> &gt;&gt;&gt; # This case does not return the length of whole series but of the batch internally</span>
<span class="sd"> ... # used.</span>
<span class="sd"> ... def length(s) -&gt; int:</span>
<span class="sd"> ... return len(s)</span>
<span class="sd"> ...</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: range(1000)})</span>
<span class="sd"> &gt;&gt;&gt; df.apply(length, axis=0) # doctest: +SKIP</span>
<span class="sd"> 0 83</span>
<span class="sd"> 1 83</span>
<span class="sd"> 2 83</span>
<span class="sd"> ...</span>
<span class="sd"> 10 83</span>
<span class="sd"> 11 83</span>
<span class="sd"> dtype: int32</span>
<span class="sd"> .. note:: this API executes the function once to infer the type which is</span>
<span class="sd"> potentially expensive, for instance, when the dataset is created after</span>
<span class="sd"> aggregations or sorting.</span>
<span class="sd"> To avoid this, specify the return type as `Series` or scalar value in ``func``,</span>
<span class="sd"> for instance, as below:</span>
<span class="sd"> &gt;&gt;&gt; def square(s) -&gt; ps.Series[np.int32]:</span>
<span class="sd"> ... return s ** 2</span>
<span class="sd"> pandas-on-Spark uses return type hints and does not try to infer the type.</span>
<span class="sd"> In case when axis is 1, it requires to specify `DataFrame` or scalar value</span>
<span class="sd"> with type hints as below:</span>
<span class="sd"> &gt;&gt;&gt; def plus_one(x) -&gt; ps.DataFrame[int, [float, float]]:</span>
<span class="sd"> ... return x + 1</span>
<span class="sd"> If the return type is specified as `DataFrame`, the output column names become</span>
<span class="sd"> `c0, c1, c2 ... cn`. These names are positionally mapped to the returned</span>
<span class="sd"> DataFrame in ``func``.</span>
<span class="sd"> To specify the column names, you can assign them in a pandas style as below:</span>
<span class="sd"> &gt;&gt;&gt; def plus_one(x) -&gt; ps.DataFrame[(&quot;index&quot;, int), [(&quot;a&quot;, float), (&quot;b&quot;, float)]]:</span>
<span class="sd"> ... return x + 1</span>
<span class="sd"> &gt;&gt;&gt; pdf = pd.DataFrame({&#39;a&#39;: [1, 2, 3], &#39;b&#39;: [3, 4, 5]})</span>
<span class="sd"> &gt;&gt;&gt; def plus_one(x) -&gt; ps.DataFrame[</span>
<span class="sd"> ... (pdf.index.name, pdf.index.dtype), zip(pdf.dtypes, pdf.columns)]:</span>
<span class="sd"> ... return x + 1</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> func : function</span>
<span class="sd"> Function to apply to each column or row.</span>
<span class="sd"> axis : {0 or &#39;index&#39;, 1 or &#39;columns&#39;}, default 0</span>
<span class="sd"> Axis along which the function is applied:</span>
<span class="sd"> * 0 or &#39;index&#39;: apply function to each column.</span>
<span class="sd"> * 1 or &#39;columns&#39;: apply function to each row.</span>
<span class="sd"> args : tuple</span>
<span class="sd"> Positional arguments to pass to `func` in addition to the</span>
<span class="sd"> array/series.</span>
<span class="sd"> **kwds</span>
<span class="sd"> Additional keyword arguments to pass as keywords arguments to</span>
<span class="sd"> `func`.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series or DataFrame</span>
<span class="sd"> Result of applying ``func`` along the given axis of the</span>
<span class="sd"> DataFrame.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.applymap : For elementwise operations.</span>
<span class="sd"> DataFrame.aggregate : Only perform aggregating type operations.</span>
<span class="sd"> DataFrame.transform : Only perform transforming type operations.</span>
<span class="sd"> Series.apply : The equivalent function for Series.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[4, 9]] * 3, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 4 9</span>
<span class="sd"> 1 4 9</span>
<span class="sd"> 2 4 9</span>
<span class="sd"> Using a numpy universal function (in this case the same as</span>
<span class="sd"> ``np.sqrt(df)``):</span>
<span class="sd"> &gt;&gt;&gt; def sqrt(x) -&gt; ps.Series[float]:</span>
<span class="sd"> ... return np.sqrt(x)</span>
<span class="sd"> ...</span>
<span class="sd"> &gt;&gt;&gt; df.apply(sqrt, axis=0)</span>
<span class="sd"> A B</span>
<span class="sd"> 0 2.0 3.0</span>
<span class="sd"> 1 2.0 3.0</span>
<span class="sd"> 2 2.0 3.0</span>
<span class="sd"> You can omit type hints and let pandas-on-Spark infer its type.</span>
<span class="sd"> &gt;&gt;&gt; df.apply(np.sqrt, axis=0)</span>
<span class="sd"> A B</span>
<span class="sd"> 0 2.0 3.0</span>
<span class="sd"> 1 2.0 3.0</span>
<span class="sd"> 2 2.0 3.0</span>
<span class="sd"> When `axis` is 1 or &#39;columns&#39;, it applies the function for each row.</span>
<span class="sd"> &gt;&gt;&gt; def summation(x) -&gt; np.int64:</span>
<span class="sd"> ... return np.sum(x)</span>
<span class="sd"> ...</span>
<span class="sd"> &gt;&gt;&gt; df.apply(summation, axis=1)</span>
<span class="sd"> 0 13</span>
<span class="sd"> 1 13</span>
<span class="sd"> 2 13</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> You can omit type hints and let pandas-on-Spark infer its type.</span>
<span class="sd"> &gt;&gt;&gt; df.apply(np.sum, axis=1)</span>
<span class="sd"> 0 13</span>
<span class="sd"> 1 13</span>
<span class="sd"> 2 13</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> &gt;&gt;&gt; df.apply(max, axis=1)</span>
<span class="sd"> 0 9</span>
<span class="sd"> 1 9</span>
<span class="sd"> 2 9</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> Returning a list-like will result in a Series</span>
<span class="sd"> &gt;&gt;&gt; df.apply(lambda x: [1, 2], axis=1)</span>
<span class="sd"> 0 [1, 2]</span>
<span class="sd"> 1 [1, 2]</span>
<span class="sd"> 2 [1, 2]</span>
<span class="sd"> dtype: object</span>
<span class="sd"> To specify the types when `axis` is &#39;1&#39;, it should use DataFrame[...]</span>
<span class="sd"> annotation. In this case, the column names are automatically generated.</span>
<span class="sd"> &gt;&gt;&gt; def identify(x) -&gt; ps.DataFrame[(&#39;index&#39;, int), [(&#39;A&#39;, np.int64), (&#39;B&#39;, np.int64)]]:</span>
<span class="sd"> ... return x</span>
<span class="sd"> ...</span>
<span class="sd"> &gt;&gt;&gt; df.apply(identify, axis=1) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> A B</span>
<span class="sd"> index</span>
<span class="sd"> 0 4 9</span>
<span class="sd"> 1 4 9</span>
<span class="sd"> 2 4 9</span>
<span class="sd"> You can also specify extra arguments.</span>
<span class="sd"> &gt;&gt;&gt; def plus_two(a, b, c) -&gt; ps.DataFrame[np.int64, [np.int64, np.int64]]:</span>
<span class="sd"> ... return a + b + c</span>
<span class="sd"> ...</span>
<span class="sd"> &gt;&gt;&gt; df.apply(plus_two, axis=1, args=(1,), c=3)</span>
<span class="sd"> c0 c1</span>
<span class="sd"> 0 8 13</span>
<span class="sd"> 1 8 13</span>
<span class="sd"> 2 8 13</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.groupby</span> <span class="kn">import</span> <span class="n">GroupBy</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">first_series</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">types</span><span class="o">.</span><span class="n">FunctionType</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">callable</span><span class="p">(</span><span class="n">func</span><span class="p">),</span> <span class="s2">&quot;the first argument should be a callable function.&quot;</span>
<span class="n">f</span> <span class="o">=</span> <span class="n">func</span>
<span class="c1"># Note that the return type hints specified here affects actual return</span>
<span class="c1"># type in Spark (e.g., infer_return_type). And MyPy does not allow</span>
<span class="c1"># redefinition of a function.</span>
<span class="n">func</span> <span class="o">=</span> <span class="k">lambda</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">f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="c1"># noqa: E731</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="n">should_return_series</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">spec</span> <span class="o">=</span> <span class="n">inspect</span><span class="o">.</span><span class="n">getfullargspec</span><span class="p">(</span><span class="n">func</span><span class="p">)</span>
<span class="n">return_sig</span> <span class="o">=</span> <span class="n">spec</span><span class="o">.</span><span class="n">annotations</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;return&quot;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="n">should_infer_schema</span> <span class="o">=</span> <span class="n">return_sig</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="n">should_retain_index</span> <span class="o">=</span> <span class="n">should_infer_schema</span>
<span class="k">def</span> <span class="nf">apply_func</span><span class="p">(</span><span class="n">pdf</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">:</span>
<span class="n">pdf_or_pser</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">args</span><span class="o">=</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwds</span><span class="p">)</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">pdf_or_pser</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="k">return</span> <span class="n">pdf_or_pser</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">pdf_or_pser</span>
<span class="n">self_applied</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="p">)</span>
<span class="n">column_labels</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">should_infer_schema</span><span class="p">:</span>
<span class="c1"># Here we execute with the first 1000 to get the return type.</span>
<span class="c1"># If the records were less than 1000, it uses pandas API directly for a shortcut.</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="s2">&quot;If the type hints is not specified for `apply`, &quot;</span>
<span class="s2">&quot;it is expensive to infer the data type internally.&quot;</span>
<span class="p">)</span>
<span class="n">limit</span> <span class="o">=</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.shortcut_limit&quot;</span><span class="p">)</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">self_applied</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">limit</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span>
<span class="n">applied</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">args</span><span class="o">=</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwds</span><span class="p">)</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="n">psser_or_psdf</span> <span class="o">=</span> <span class="n">ps</span><span class="o">.</span><span class="n">from_pandas</span><span class="p">(</span><span class="n">applied</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="n">limit</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psser_or_psdf</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">psser_or_psdf</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">psser_or_psdf</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="n">should_return_series</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">psser_or_psdf</span><span class="o">.</span><span class="n">_psdf</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="p">[</span><span class="n">field</span><span class="o">.</span><span class="n">normalize_spark_type</span><span class="p">()</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">]</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="p">[</span><span class="n">field</span><span class="o">.</span><span class="n">normalize_spark_type</span><span class="p">()</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">]</span>
<span class="n">return_schema</span> <span class="o">=</span> <span class="n">StructType</span><span class="p">([</span><span class="n">field</span><span class="o">.</span><span class="n">struct_field</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="n">index_fields</span> <span class="o">+</span> <span class="n">data_fields</span><span class="p">])</span>
<span class="n">output_func</span> <span class="o">=</span> <span class="n">GroupBy</span><span class="o">.</span><span class="n">_make_pandas_df_builder_func</span><span class="p">(</span>
<span class="n">self_applied</span><span class="p">,</span> <span class="n">apply_func</span><span class="p">,</span> <span class="n">return_schema</span><span class="p">,</span> <span class="n">retain_index</span><span class="o">=</span><span class="n">should_retain_index</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">self_applied</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">to_internal_spark_frame</span><span class="o">.</span><span class="n">mapInPandas</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">iterator</span><span class="p">:</span> <span class="nb">map</span><span class="p">(</span><span class="n">output_func</span><span class="p">,</span> <span class="n">iterator</span><span class="p">),</span> <span class="n">schema</span><span class="o">=</span><span class="n">return_schema</span>
<span class="p">)</span>
<span class="c1"># If schema is inferred, we can restore indexes too.</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span> <span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span><span class="p">,</span> <span class="n">data_fields</span><span class="o">=</span><span class="n">data_fields</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">return_type</span> <span class="o">=</span> <span class="n">infer_return_type</span><span class="p">(</span><span class="n">func</span><span class="p">)</span>
<span class="n">require_index_axis</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">return_type</span><span class="p">,</span> <span class="n">SeriesType</span><span class="p">)</span>
<span class="n">require_column_axis</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">return_type</span><span class="p">,</span> <span class="n">DataFrameType</span><span class="p">)</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">require_index_axis</span><span class="p">:</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;The given function should specify a scalar or a series as its type &quot;</span>
<span class="s2">&quot;hints when axis is 0 or &#39;index&#39;; however, the return type &quot;</span>
<span class="s2">&quot;was </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="n">return_sig</span>
<span class="p">)</span>
<span class="n">dtype</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">SeriesType</span><span class="p">,</span> <span class="n">return_type</span><span class="p">)</span><span class="o">.</span><span class="n">dtype</span>
<span class="n">spark_type</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">SeriesType</span><span class="p">,</span> <span class="n">return_type</span><span class="p">)</span><span class="o">.</span><span class="n">spark_type</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">InternalField</span><span class="p">(</span>
<span class="n">dtype</span><span class="o">=</span><span class="n">dtype</span><span class="p">,</span> <span class="n">struct_field</span><span class="o">=</span><span class="n">StructField</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">name</span><span class="p">,</span> <span class="n">dataType</span><span class="o">=</span><span class="n">spark_type</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">name</span> <span class="ow">in</span> <span class="n">self_applied</span><span class="o">.</span><span class="n">columns</span>
<span class="p">]</span>
<span class="n">return_schema</span> <span class="o">=</span> <span class="n">StructType</span><span class="p">([</span><span class="n">field</span><span class="o">.</span><span class="n">struct_field</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="n">data_fields</span><span class="p">])</span>
<span class="k">elif</span> <span class="n">require_column_axis</span><span class="p">:</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;The given function should specify a scalar or a frame as its type &quot;</span>
<span class="s2">&quot;hints when axis is 1 or &#39;column&#39;; however, the return type &quot;</span>
<span class="s2">&quot;was </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="n">return_sig</span>
<span class="p">)</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">DataFrameType</span><span class="p">,</span> <span class="n">return_type</span><span class="p">)</span><span class="o">.</span><span class="n">index_fields</span>
<span class="n">should_retain_index</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">index_fields</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">DataFrameType</span><span class="p">,</span> <span class="n">return_type</span><span class="p">)</span><span class="o">.</span><span class="n">data_fields</span>
<span class="n">return_schema</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">DataFrameType</span><span class="p">,</span> <span class="n">return_type</span><span class="p">)</span><span class="o">.</span><span class="n">spark_type</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># any axis is fine.</span>
<span class="n">should_return_series</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">spark_type</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">ScalarType</span><span class="p">,</span> <span class="n">return_type</span><span class="p">)</span><span class="o">.</span><span class="n">spark_type</span>
<span class="n">dtype</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">ScalarType</span><span class="p">,</span> <span class="n">return_type</span><span class="p">)</span><span class="o">.</span><span class="n">dtype</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">InternalField</span><span class="p">(</span>
<span class="n">dtype</span><span class="o">=</span><span class="n">dtype</span><span class="p">,</span>
<span class="n">struct_field</span><span class="o">=</span><span class="n">StructField</span><span class="p">(</span>
<span class="n">name</span><span class="o">=</span><span class="n">SPARK_DEFAULT_SERIES_NAME</span><span class="p">,</span> <span class="n">dataType</span><span class="o">=</span><span class="n">spark_type</span>
<span class="p">),</span>
<span class="p">)</span>
<span class="p">]</span>
<span class="n">return_schema</span> <span class="o">=</span> <span class="n">StructType</span><span class="p">([</span><span class="n">field</span><span class="o">.</span><span class="n">struct_field</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="n">data_fields</span><span class="p">])</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span><span class="kc">None</span><span class="p">]</span>
<span class="n">output_func</span> <span class="o">=</span> <span class="n">GroupBy</span><span class="o">.</span><span class="n">_make_pandas_df_builder_func</span><span class="p">(</span>
<span class="n">self_applied</span><span class="p">,</span> <span class="n">apply_func</span><span class="p">,</span> <span class="n">return_schema</span><span class="p">,</span> <span class="n">retain_index</span><span class="o">=</span><span class="n">should_retain_index</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">self_applied</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">to_internal_spark_frame</span><span class="o">.</span><span class="n">mapInPandas</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">iterator</span><span class="p">:</span> <span class="nb">map</span><span class="p">(</span><span class="n">output_func</span><span class="p">,</span> <span class="n">iterator</span><span class="p">),</span> <span class="n">schema</span><span class="o">=</span><span class="n">return_schema</span>
<span class="p">)</span>
<span class="n">index_spark_columns</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">index_names</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Optional</span><span class="p">[</span><span class="n">Tuple</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="o">...</span><span class="p">]]]]</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">should_retain_index</span><span class="p">:</span>
<span class="n">index_spark_columns</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">index_field</span><span class="o">.</span><span class="n">struct_field</span><span class="o">.</span><span class="n">name</span><span class="p">)</span> <span class="k">for</span> <span class="n">index_field</span> <span class="ow">in</span> <span class="n">index_fields</span>
<span class="p">]</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">any</span><span class="p">(</span>
<span class="p">[</span>
<span class="n">SPARK_INDEX_NAME_PATTERN</span><span class="o">.</span><span class="n">match</span><span class="p">(</span><span class="n">index_field</span><span class="o">.</span><span class="n">struct_field</span><span class="o">.</span><span class="n">name</span><span class="p">)</span>
<span class="k">for</span> <span class="n">index_field</span> <span class="ow">in</span> <span class="n">index_fields</span>
<span class="p">]</span>
<span class="p">):</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="p">[(</span><span class="n">index_field</span><span class="o">.</span><span class="n">struct_field</span><span class="o">.</span><span class="n">name</span><span class="p">,)</span> <span class="k">for</span> <span class="n">index_field</span> <span class="ow">in</span> <span class="n">index_fields</span><span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index_names</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="n">index_spark_columns</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span><span class="p">,</span>
<span class="n">data_fields</span><span class="o">=</span><span class="n">data_fields</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">result</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">if</span> <span class="n">should_return_series</span><span class="p">:</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">result</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="DataFrame.transform"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.transform.html#pyspark.pandas.DataFrame.transform">[docs]</a> <span class="k">def</span> <span class="nf">transform</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">:</span> <span class="n">Callable</span><span class="p">[</span><span class="o">...</span><span class="p">,</span> <span class="s2">&quot;Series&quot;</span><span class="p">],</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Call ``func`` on self producing a Series with transformed values</span>
<span class="sd"> and that has the same length as its input.</span>
<span class="sd"> See also `Transform and apply a function</span>
<span class="sd"> &lt;https://spark.apache.org/docs/latest/api/python/user_guide/pandas_on_spark/transform_apply.html&gt;`_.</span>
<span class="sd"> .. note:: this API executes the function once to infer the type which is</span>
<span class="sd"> potentially expensive, for instance, when the dataset is created after</span>
<span class="sd"> aggregations or sorting.</span>
<span class="sd"> To avoid this, specify return type in ``func``, for instance, as below:</span>
<span class="sd"> &gt;&gt;&gt; def square(x) -&gt; ps.Series[np.int32]:</span>
<span class="sd"> ... return x ** 2</span>
<span class="sd"> pandas-on-Spark uses return type hints and does not try to infer the type.</span>
<span class="sd"> .. note:: the series within ``func`` is actually multiple pandas series as the</span>
<span class="sd"> segments of the whole pandas-on-Spark series; therefore, the length of each series</span>
<span class="sd"> is not guaranteed. As an example, an aggregation against each series</span>
<span class="sd"> does work as a global aggregation but an aggregation of each segment. See</span>
<span class="sd"> below:</span>
<span class="sd"> &gt;&gt;&gt; def func(x) -&gt; ps.Series[np.int32]:</span>
<span class="sd"> ... return x + sum(x)</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> func : function</span>
<span class="sd"> Function to use for transforming the data. It must work when pandas Series</span>
<span class="sd"> is passed.</span>
<span class="sd"> axis : int, default 0 or &#39;index&#39;</span>
<span class="sd"> Can only be set to 0 now.</span>
<span class="sd"> *args</span>
<span class="sd"> Positional arguments to pass to func.</span>
<span class="sd"> **kwargs</span>
<span class="sd"> Keyword arguments to pass to func.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> A DataFrame that must have the same length as self.</span>
<span class="sd"> Raises</span>
<span class="sd"> ------</span>
<span class="sd"> Exception : If the returned DataFrame has a different length than self.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.aggregate : Only perform aggregating type operations.</span>
<span class="sd"> DataFrame.apply : Invoke function on DataFrame.</span>
<span class="sd"> Series.transform : The equivalent function for Series.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: range(3), &#39;B&#39;: range(1, 4)}, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 1</span>
<span class="sd"> 1 1 2</span>
<span class="sd"> 2 2 3</span>
<span class="sd"> &gt;&gt;&gt; def square(x) -&gt; ps.Series[np.int32]:</span>
<span class="sd"> ... return x ** 2</span>
<span class="sd"> &gt;&gt;&gt; df.transform(square)</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 1</span>
<span class="sd"> 1 1 4</span>
<span class="sd"> 2 4 9</span>
<span class="sd"> You can omit type hints and let pandas-on-Spark infer its type.</span>
<span class="sd"> &gt;&gt;&gt; df.transform(lambda x: x ** 2)</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 1</span>
<span class="sd"> 1 1 4</span>
<span class="sd"> 2 4 9</span>
<span class="sd"> For multi-index columns:</span>
<span class="sd"> &gt;&gt;&gt; df.columns = [(&#39;X&#39;, &#39;A&#39;), (&#39;X&#39;, &#39;B&#39;)]</span>
<span class="sd"> &gt;&gt;&gt; df.transform(square) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> X</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 1</span>
<span class="sd"> 1 1 4</span>
<span class="sd"> 2 4 9</span>
<span class="sd"> &gt;&gt;&gt; (df * -1).transform(abs) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> X</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 1</span>
<span class="sd"> 1 1 2</span>
<span class="sd"> 2 2 3</span>
<span class="sd"> You can also specify extra arguments.</span>
<span class="sd"> &gt;&gt;&gt; def calculation(x, y, z) -&gt; ps.Series[int]:</span>
<span class="sd"> ... return x ** y + z</span>
<span class="sd"> &gt;&gt;&gt; df.transform(calculation, y=10, z=20) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> X</span>
<span class="sd"> A B</span>
<span class="sd"> 0 20 21</span>
<span class="sd"> 1 21 1044</span>
<span class="sd"> 2 1044 59069</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">types</span><span class="o">.</span><span class="n">FunctionType</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">callable</span><span class="p">(</span><span class="n">func</span><span class="p">),</span> <span class="s2">&quot;the first argument should be a callable function.&quot;</span>
<span class="n">f</span> <span class="o">=</span> <span class="n">func</span>
<span class="c1"># Note that the return type hints specified here affects actual return</span>
<span class="c1"># type in Spark (e.g., infer_return_type). And, MyPy does not allow</span>
<span class="c1"># redefinition of a function.</span>
<span class="n">func</span> <span class="o">=</span> <span class="k">lambda</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">f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="c1"># noqa: E731</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</span><span class="p">)</span>
<span class="n">spec</span> <span class="o">=</span> <span class="n">inspect</span><span class="o">.</span><span class="n">getfullargspec</span><span class="p">(</span><span class="n">func</span><span class="p">)</span>
<span class="n">return_sig</span> <span class="o">=</span> <span class="n">spec</span><span class="o">.</span><span class="n">annotations</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;return&quot;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="n">should_infer_schema</span> <span class="o">=</span> <span class="n">return_sig</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">should_infer_schema</span><span class="p">:</span>
<span class="c1"># Here we execute with the first 1000 to get the return type.</span>
<span class="c1"># If the records were less than 1000, it uses pandas API directly for a shortcut.</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="s2">&quot;If the type hints is not specified for `transform`, &quot;</span>
<span class="s2">&quot;it is expensive to infer the data type internally.&quot;</span>
<span class="p">)</span>
<span class="n">limit</span> <span class="o">=</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.shortcut_limit&quot;</span><span class="p">)</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">limit</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span>
<span class="n">transformed</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">transform</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">axis</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">transformed</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="n">limit</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span>
<span class="n">applied</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">input_label</span><span class="p">,</span> <span class="n">output_label</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">):</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">input_label</span><span class="p">)</span>
<span class="n">field</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">field_for</span><span class="p">(</span><span class="n">output_label</span><span class="p">)</span><span class="o">.</span><span class="n">normalize_spark_type</span><span class="p">()</span>
<span class="n">data_fields</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">field</span><span class="p">)</span>
<span class="n">return_schema</span> <span class="o">=</span> <span class="n">field</span><span class="o">.</span><span class="n">spark_type</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">psser</span><span class="o">.</span><span class="n">pandas_on_spark</span><span class="o">.</span><span class="n">_transform_batch</span><span class="p">(</span>
<span class="n">func</span><span class="o">=</span><span class="k">lambda</span> <span class="n">c</span><span class="p">:</span> <span class="n">func</span><span class="p">(</span><span class="n">c</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">return_type</span><span class="o">=</span><span class="n">SeriesType</span><span class="p">(</span><span class="n">field</span><span class="o">.</span><span class="n">dtype</span><span class="p">,</span> <span class="n">return_schema</span><span class="p">),</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">applied</span><span class="p">,</span> <span class="n">data_fields</span><span class="o">=</span><span class="n">data_fields</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">pandas_on_spark</span><span class="o">.</span><span class="n">transform_batch</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.pop"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.pop.html#pyspark.pandas.DataFrame.pop">[docs]</a> <span class="k">def</span> <span class="nf">pop</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">item</span><span class="p">:</span> <span class="n">Name</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return item and drop from frame. Raise KeyError if not found.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> item : str</span>
<span class="sd"> Label of column to be popped.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(&#39;falcon&#39;, &#39;bird&#39;, 389.0),</span>
<span class="sd"> ... (&#39;parrot&#39;, &#39;bird&#39;, 24.0),</span>
<span class="sd"> ... (&#39;lion&#39;, &#39;mammal&#39;, 80.5),</span>
<span class="sd"> ... (&#39;monkey&#39;,&#39;mammal&#39;, np.nan)],</span>
<span class="sd"> ... columns=(&#39;name&#39;, &#39;class&#39;, &#39;max_speed&#39;))</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> name class max_speed</span>
<span class="sd"> 0 falcon bird 389.0</span>
<span class="sd"> 1 parrot bird 24.0</span>
<span class="sd"> 2 lion mammal 80.5</span>
<span class="sd"> 3 monkey mammal NaN</span>
<span class="sd"> &gt;&gt;&gt; df.pop(&#39;class&#39;)</span>
<span class="sd"> 0 bird</span>
<span class="sd"> 1 bird</span>
<span class="sd"> 2 mammal</span>
<span class="sd"> 3 mammal</span>
<span class="sd"> Name: class, dtype: object</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> name max_speed</span>
<span class="sd"> 0 falcon 389.0</span>
<span class="sd"> 1 parrot 24.0</span>
<span class="sd"> 2 lion 80.5</span>
<span class="sd"> 3 monkey NaN</span>
<span class="sd"> Also support for MultiIndex</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(&#39;falcon&#39;, &#39;bird&#39;, 389.0),</span>
<span class="sd"> ... (&#39;parrot&#39;, &#39;bird&#39;, 24.0),</span>
<span class="sd"> ... (&#39;lion&#39;, &#39;mammal&#39;, 80.5),</span>
<span class="sd"> ... (&#39;monkey&#39;,&#39;mammal&#39;, np.nan)],</span>
<span class="sd"> ... columns=(&#39;name&#39;, &#39;class&#39;, &#39;max_speed&#39;))</span>
<span class="sd"> &gt;&gt;&gt; columns = [(&#39;a&#39;, &#39;name&#39;), (&#39;a&#39;, &#39;class&#39;), (&#39;b&#39;, &#39;max_speed&#39;)]</span>
<span class="sd"> &gt;&gt;&gt; df.columns = pd.MultiIndex.from_tuples(columns)</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> a b</span>
<span class="sd"> name class max_speed</span>
<span class="sd"> 0 falcon bird 389.0</span>
<span class="sd"> 1 parrot bird 24.0</span>
<span class="sd"> 2 lion mammal 80.5</span>
<span class="sd"> 3 monkey mammal NaN</span>
<span class="sd"> &gt;&gt;&gt; df.pop(&#39;a&#39;)</span>
<span class="sd"> name class</span>
<span class="sd"> 0 falcon bird</span>
<span class="sd"> 1 parrot bird</span>
<span class="sd"> 2 lion mammal</span>
<span class="sd"> 3 monkey mammal</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> b</span>
<span class="sd"> max_speed</span>
<span class="sd"> 0 389.0</span>
<span class="sd"> 1 24.0</span>
<span class="sd"> 2 80.5</span>
<span class="sd"> 3 NaN</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="n">item</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">columns</span><span class="o">=</span><span class="n">item</span><span class="p">)</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span></div>
<span class="c1"># TODO(SPARK-46158): add axis parameter can work when &#39;1&#39; or &#39;columns&#39;</span>
<div class="viewcode-block" id="DataFrame.xs"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.xs.html#pyspark.pandas.DataFrame.xs">[docs]</a> <span class="k">def</span> <span class="nf">xs</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">:</span> <span class="n">Name</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="n">level</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">DataFrameOrSeries</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return cross-section from the DataFrame.</span>
<span class="sd"> This method takes a `key` argument to select data at a particular</span>
<span class="sd"> level of a MultiIndex.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> key : label or tuple of label</span>
<span class="sd"> Label contained in the index, or partially in a MultiIndex.</span>
<span class="sd"> axis : 0 or &#39;index&#39;, default 0</span>
<span class="sd"> Axis to retrieve cross-section on.</span>
<span class="sd"> currently only support 0 or &#39;index&#39;</span>
<span class="sd"> level : object, defaults to first n levels (n=1 or len(key))</span>
<span class="sd"> In case of a key partially contained in a MultiIndex, indicate</span>
<span class="sd"> which levels are used. Levels can be referred by label or position.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame or Series</span>
<span class="sd"> Cross-section from the original DataFrame</span>
<span class="sd"> corresponding to the selected index levels.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.loc : Access a group of rows and columns</span>
<span class="sd"> by label(s) or a boolean array.</span>
<span class="sd"> DataFrame.iloc : Purely integer-location based indexing</span>
<span class="sd"> for selection by position.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; d = {&#39;num_legs&#39;: [4, 4, 2, 2],</span>
<span class="sd"> ... &#39;num_wings&#39;: [0, 0, 2, 2],</span>
<span class="sd"> ... &#39;class&#39;: [&#39;mammal&#39;, &#39;mammal&#39;, &#39;mammal&#39;, &#39;bird&#39;],</span>
<span class="sd"> ... &#39;animal&#39;: [&#39;cat&#39;, &#39;dog&#39;, &#39;bat&#39;, &#39;penguin&#39;],</span>
<span class="sd"> ... &#39;locomotion&#39;: [&#39;walks&#39;, &#39;walks&#39;, &#39;flies&#39;, &#39;walks&#39;]}</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(data=d)</span>
<span class="sd"> &gt;&gt;&gt; df = df.set_index([&#39;class&#39;, &#39;animal&#39;, &#39;locomotion&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs num_wings</span>
<span class="sd"> class animal locomotion</span>
<span class="sd"> mammal cat walks 4 0</span>
<span class="sd"> dog walks 4 0</span>
<span class="sd"> bat flies 2 2</span>
<span class="sd"> bird penguin walks 2 2</span>
<span class="sd"> Get values at specified index</span>
<span class="sd"> &gt;&gt;&gt; df.xs(&#39;mammal&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs num_wings</span>
<span class="sd"> animal locomotion</span>
<span class="sd"> cat walks 4 0</span>
<span class="sd"> dog walks 4 0</span>
<span class="sd"> bat flies 2 2</span>
<span class="sd"> Get values at several indexes</span>
<span class="sd"> &gt;&gt;&gt; df.xs((&#39;mammal&#39;, &#39;dog&#39;)) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs num_wings</span>
<span class="sd"> locomotion</span>
<span class="sd"> walks 4 0</span>
<span class="sd"> &gt;&gt;&gt; df.xs((&#39;mammal&#39;, &#39;dog&#39;, &#39;walks&#39;)) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs 4</span>
<span class="sd"> num_wings 0</span>
<span class="sd"> Name: (mammal, dog, walks), dtype: int64</span>
<span class="sd"> Get values at specified index and level</span>
<span class="sd"> &gt;&gt;&gt; df.xs(&#39;cat&#39;, level=1) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs num_wings</span>
<span class="sd"> class locomotion</span>
<span class="sd"> mammal walks 4 0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">first_series</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">key</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;&#39;key&#39; should be a scalar value or tuple that contains scalar values&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">key</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="n">key</span><span class="p">)</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">key</span><span class="p">):</span>
<span class="n">key</span> <span class="o">=</span> <span class="p">(</span><span class="n">key</span><span class="p">,)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="o">&gt;</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span>
<span class="s2">&quot;Key length (</span><span class="si">{}</span><span class="s2">) exceeds index depth (</span><span class="si">{}</span><span class="s2">)&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="nb">len</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">_internal</span><span class="o">.</span><span class="n">index_level</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">level</span> <span class="o">=</span> <span class="mi">0</span>
<span class="n">rows</span> <span class="o">=</span> <span class="p">[</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="n">lvl</span><span class="p">]</span> <span class="o">==</span> <span class="n">index</span> <span class="k">for</span> <span class="n">lvl</span><span class="p">,</span> <span class="n">index</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">level</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">reduce</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">:</span> <span class="n">x</span> <span class="o">&amp;</span> <span class="n">y</span><span class="p">,</span> <span class="n">rows</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="o">==</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">:</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="mi">2</span><span class="p">)</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="n">key</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</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">psdf</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">pdf</span><span class="o">.</span><span class="n">transpose</span><span class="p">()))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">index_spark_columns</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">[:</span><span class="n">level</span><span class="p">]</span>
<span class="o">+</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="n">level</span> <span class="o">+</span> <span class="nb">len</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="p">:]</span>
<span class="p">)</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">[:</span><span class="n">level</span><span class="p">]</span> <span class="o">+</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">[</span><span class="n">level</span> <span class="o">+</span> <span class="nb">len</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="p">:]</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">[:</span><span class="n">level</span><span class="p">]</span> <span class="o">+</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">[</span><span class="n">level</span> <span class="o">+</span> <span class="nb">len</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="p">:]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="n">index_spark_columns</span><span class="p">,</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index_names</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span><span class="p">,</span>
<span class="p">)</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.between_time"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.between_time.html#pyspark.pandas.DataFrame.between_time">[docs]</a> <span class="k">def</span> <span class="nf">between_time</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">start_time</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">datetime</span><span class="o">.</span><span class="n">time</span><span class="p">,</span> <span class="nb">str</span><span class="p">],</span>
<span class="n">end_time</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">datetime</span><span class="o">.</span><span class="n">time</span><span class="p">,</span> <span class="nb">str</span><span class="p">],</span>
<span class="n">inclusive</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;both&quot;</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Select values between particular times of the day (example: 9:00-9:30 AM).</span>
<span class="sd"> By setting ``start_time`` to be later than ``end_time``,</span>
<span class="sd"> you can get the times that are *not* between the two times.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> start_time : datetime.time or str</span>
<span class="sd"> Initial time as a time filter limit.</span>
<span class="sd"> end_time : datetime.time or str</span>
<span class="sd"> End time as a time filter limit.</span>
<span class="sd"> inclusive : {&quot;both&quot;, &quot;neither&quot;, &quot;left&quot;, &quot;right&quot;}, default &quot;both&quot;</span>
<span class="sd"> Include boundaries; whether to set each bound as closed or open.</span>
<span class="sd"> .. versionadded:: 4.0.0</span>
<span class="sd"> axis : {0 or &#39;index&#39;, 1 or &#39;columns&#39;}, default 0</span>
<span class="sd"> Determine range time on index or columns value.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Data from the original object filtered to the specified dates range.</span>
<span class="sd"> Raises</span>
<span class="sd"> ------</span>
<span class="sd"> TypeError</span>
<span class="sd"> If the index is not a :class:`DatetimeIndex`</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> at_time : Select values at a particular time of the day.</span>
<span class="sd"> first : Select initial periods of time series based on a date offset.</span>
<span class="sd"> last : Select final periods of time series based on a date offset.</span>
<span class="sd"> DatetimeIndex.indexer_between_time : Get just the index locations for</span>
<span class="sd"> values between particular times of the day.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; idx = pd.date_range(&#39;2018-04-09&#39;, periods=4, freq=&#39;1D20min&#39;)</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;A&#39;: [1, 2, 3, 4]}, index=idx)</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-09 00:00:00 1</span>
<span class="sd"> 2018-04-10 00:20:00 2</span>
<span class="sd"> 2018-04-11 00:40:00 3</span>
<span class="sd"> 2018-04-12 01:00:00 4</span>
<span class="sd"> &gt;&gt;&gt; psdf.between_time(&#39;0:15&#39;, &#39;0:45&#39;) # doctest: +SKIP</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-10 00:20:00 2</span>
<span class="sd"> 2018-04-11 00:40:00 3</span>
<span class="sd"> You get the times that are *not* between two times by setting</span>
<span class="sd"> ``start_time`` later than ``end_time``:</span>
<span class="sd"> &gt;&gt;&gt; psdf.between_time(&#39;0:45&#39;, &#39;0:15&#39;) # doctest: +SKIP</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-09 00:00:00 1</span>
<span class="sd"> 2018-04-12 01:00:00 4</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;between_time currently only works for axis=0&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">DatetimeIndex</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Index must be DatetimeIndex&quot;</span><span class="p">)</span>
<span class="n">allowed_inclusive_values</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;left&quot;</span><span class="p">,</span> <span class="s2">&quot;right&quot;</span><span class="p">,</span> <span class="s2">&quot;both&quot;</span><span class="p">,</span> <span class="s2">&quot;neither&quot;</span><span class="p">]</span>
<span class="k">if</span> <span class="n">inclusive</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">allowed_inclusive_values</span><span class="p">:</span>
<span class="k">raise</span> <span class="n">PySparkValueError</span><span class="p">(</span>
<span class="n">error_class</span><span class="o">=</span><span class="s2">&quot;VALUE_NOT_ALLOWED&quot;</span><span class="p">,</span>
<span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span>
<span class="s2">&quot;arg_name&quot;</span><span class="p">:</span> <span class="s2">&quot;inclusive&quot;</span><span class="p">,</span>
<span class="s2">&quot;allowed_values&quot;</span><span class="p">:</span> <span class="nb">str</span><span class="p">(</span><span class="n">allowed_inclusive_values</span><span class="p">),</span>
<span class="p">},</span>
<span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">psdf</span><span class="p">,</span> <span class="s2">&quot;__index_name__&quot;</span><span class="p">)</span>
<span class="n">return_types</span> <span class="o">=</span> <span class="p">[</span><span class="n">psdf</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">dtype</span><span class="p">]</span> <span class="o">+</span> <span class="nb">list</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">dtypes</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">pandas_between_time</span><span class="p">(</span> <span class="c1"># type: ignore[no-untyped-def]</span>
<span class="n">pdf</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">ps</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">[</span><span class="n">return_types</span><span class="p">]:</span> <span class="c1"># type: ignore[valid-type]</span>
<span class="k">return</span> <span class="n">pdf</span><span class="o">.</span><span class="n">between_time</span><span class="p">(</span><span class="n">start_time</span><span class="p">,</span> <span class="n">end_time</span><span class="p">,</span> <span class="n">inclusive</span><span class="p">)</span><span class="o">.</span><span class="n">reset_index</span><span class="p">()</span>
<span class="c1"># apply_batch will remove the index of the pandas-on-Spark DataFrame and attach a</span>
<span class="c1"># default index, which will never be used. Use &quot;distributed&quot; index as a dummy to</span>
<span class="c1"># avoid overhead.</span>
<span class="k">with</span> <span class="n">option_context</span><span class="p">(</span><span class="s2">&quot;compute.default_index_type&quot;</span><span class="p">,</span> <span class="s2">&quot;distributed&quot;</span><span class="p">):</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">pandas_on_spark</span><span class="o">.</span><span class="n">apply_batch</span><span class="p">(</span><span class="n">pandas_between_time</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">[:</span><span class="mi">1</span><span class="p">],</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">[:</span><span class="mi">1</span><span class="p">],</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">[</span><span class="mi">1</span><span class="p">:],</span>
<span class="n">data_fields</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">[</span><span class="mi">1</span><span class="p">:],</span>
<span class="p">)</span>
<span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46159): implement axis=1</span>
<div class="viewcode-block" id="DataFrame.at_time"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.at_time.html#pyspark.pandas.DataFrame.at_time">[docs]</a> <span class="k">def</span> <span class="nf">at_time</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">time</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">datetime</span><span class="o">.</span><span class="n">time</span><span class="p">,</span> <span class="nb">str</span><span class="p">],</span> <span class="n">asof</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Select values at particular time of day (example: 9:30AM).</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> time : datetime.time or str</span>
<span class="sd"> axis : {0 or &#39;index&#39;, 1 or &#39;columns&#39;}, default 0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Raises</span>
<span class="sd"> ------</span>
<span class="sd"> TypeError</span>
<span class="sd"> If the index is not a :class:`DatetimeIndex`</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> between_time : Select values between particular times of the day.</span>
<span class="sd"> DatetimeIndex.indexer_at_time : Get just the index locations for</span>
<span class="sd"> values at particular time of the day.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; idx = pd.date_range(&#39;2018-04-09&#39;, periods=4, freq=&#39;12H&#39;)</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;A&#39;: [1, 2, 3, 4]}, index=idx)</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-09 00:00:00 1</span>
<span class="sd"> 2018-04-09 12:00:00 2</span>
<span class="sd"> 2018-04-10 00:00:00 3</span>
<span class="sd"> 2018-04-10 12:00:00 4</span>
<span class="sd"> &gt;&gt;&gt; psdf.at_time(&#39;12:00&#39;)</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-09 12:00:00 2</span>
<span class="sd"> 2018-04-10 12:00:00 4</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">asof</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;&#39;asof&#39; argument is not supported&quot;</span><span class="p">)</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;at_time currently only works for axis=0&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">DatetimeIndex</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Index must be DatetimeIndex&quot;</span><span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">psdf</span><span class="p">,</span> <span class="s2">&quot;__index_name__&quot;</span><span class="p">)</span>
<span class="n">return_types</span> <span class="o">=</span> <span class="p">[</span><span class="n">psdf</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">dtype</span><span class="p">]</span> <span class="o">+</span> <span class="nb">list</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">dtypes</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">pandas_at_time</span><span class="p">(</span> <span class="c1"># type: ignore[no-untyped-def]</span>
<span class="n">pdf</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">ps</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">[</span><span class="n">return_types</span><span class="p">]:</span> <span class="c1"># type: ignore[valid-type]</span>
<span class="k">return</span> <span class="n">pdf</span><span class="o">.</span><span class="n">at_time</span><span class="p">(</span><span class="n">time</span><span class="p">,</span> <span class="n">asof</span><span class="p">,</span> <span class="n">axis</span><span class="p">)</span><span class="o">.</span><span class="n">reset_index</span><span class="p">()</span>
<span class="c1"># apply_batch will remove the index of the pandas-on-Spark DataFrame and attach</span>
<span class="c1"># a default index, which will never be used. Use &quot;distributed&quot; index as a dummy</span>
<span class="c1"># to avoid overhead.</span>
<span class="k">with</span> <span class="n">option_context</span><span class="p">(</span><span class="s2">&quot;compute.default_index_type&quot;</span><span class="p">,</span> <span class="s2">&quot;distributed&quot;</span><span class="p">):</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">pandas_on_spark</span><span class="o">.</span><span class="n">apply_batch</span><span class="p">(</span><span class="n">pandas_at_time</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">[:</span><span class="mi">1</span><span class="p">],</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">[:</span><span class="mi">1</span><span class="p">],</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">[</span><span class="mi">1</span><span class="p">:],</span>
<span class="n">data_fields</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">[</span><span class="mi">1</span><span class="p">:],</span>
<span class="p">)</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.where"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.where.html#pyspark.pandas.DataFrame.where">[docs]</a> <span class="k">def</span> <span class="nf">where</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">cond</span><span class="p">:</span> <span class="n">DataFrameOrSeries</span><span class="p">,</span>
<span class="n">other</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">DataFrameOrSeries</span><span class="p">,</span> <span class="n">Any</span><span class="p">]</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">nan</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Replace values where the condition is False.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> cond : boolean DataFrame</span>
<span class="sd"> Where cond is True, keep the original value. Where False,</span>
<span class="sd"> replace with corresponding value from other.</span>
<span class="sd"> other : scalar, DataFrame</span>
<span class="sd"> Entries where cond is False are replaced with corresponding value from other.</span>
<span class="sd"> axis : int, default None</span>
<span class="sd"> Can only be set to 0 now for compatibility with pandas.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.pandas.config import set_option, reset_option</span>
<span class="sd"> &gt;&gt;&gt; set_option(&quot;compute.ops_on_diff_frames&quot;, True)</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame({&#39;A&#39;: [0, 1, 2, 3, 4], &#39;B&#39;:[100, 200, 300, 400, 500]})</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame({&#39;A&#39;: [0, -1, -2, -3, -4], &#39;B&#39;:[-100, -200, -300, -400, -500]})</span>
<span class="sd"> &gt;&gt;&gt; df1</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 100</span>
<span class="sd"> 1 1 200</span>
<span class="sd"> 2 2 300</span>
<span class="sd"> 3 3 400</span>
<span class="sd"> 4 4 500</span>
<span class="sd"> &gt;&gt;&gt; df2</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 -100</span>
<span class="sd"> 1 -1 -200</span>
<span class="sd"> 2 -2 -300</span>
<span class="sd"> 3 -3 -400</span>
<span class="sd"> 4 -4 -500</span>
<span class="sd"> &gt;&gt;&gt; df1.where(df1 &gt; 0).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 NaN 100.0</span>
<span class="sd"> 1 1.0 200.0</span>
<span class="sd"> 2 2.0 300.0</span>
<span class="sd"> 3 3.0 400.0</span>
<span class="sd"> 4 4.0 500.0</span>
<span class="sd"> &gt;&gt;&gt; df1.where(df1 &gt; 1, 10).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 10 100</span>
<span class="sd"> 1 10 200</span>
<span class="sd"> 2 2 300</span>
<span class="sd"> 3 3 400</span>
<span class="sd"> 4 4 500</span>
<span class="sd"> &gt;&gt;&gt; df1.where(df1 &gt; 1, df1 + 100).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 100 100</span>
<span class="sd"> 1 101 200</span>
<span class="sd"> 2 2 300</span>
<span class="sd"> 3 3 400</span>
<span class="sd"> 4 4 500</span>
<span class="sd"> &gt;&gt;&gt; df1.where(df1 &gt; 1, df2).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 100</span>
<span class="sd"> 1 -1 200</span>
<span class="sd"> 2 2 300</span>
<span class="sd"> 3 3 400</span>
<span class="sd"> 4 4 500</span>
<span class="sd"> When the column name of cond is different from self, it treats all values are False</span>
<span class="sd"> &gt;&gt;&gt; cond = ps.DataFrame({&#39;C&#39;: [0, -1, -2, -3, -4], &#39;D&#39;:[4, 3, 2, 1, 0]}) % 3 == 0</span>
<span class="sd"> &gt;&gt;&gt; cond</span>
<span class="sd"> C D</span>
<span class="sd"> 0 True False</span>
<span class="sd"> 1 False True</span>
<span class="sd"> 2 False False</span>
<span class="sd"> 3 True False</span>
<span class="sd"> 4 False True</span>
<span class="sd"> &gt;&gt;&gt; df1.where(cond).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 NaN NaN</span>
<span class="sd"> 1 NaN NaN</span>
<span class="sd"> 2 NaN NaN</span>
<span class="sd"> 3 NaN NaN</span>
<span class="sd"> 4 NaN NaN</span>
<span class="sd"> When the type of cond is Series, it just check boolean regardless of column name</span>
<span class="sd"> &gt;&gt;&gt; cond = ps.Series([1, 2]) &gt; 1</span>
<span class="sd"> &gt;&gt;&gt; cond</span>
<span class="sd"> 0 False</span>
<span class="sd"> 1 True</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> &gt;&gt;&gt; df1.where(cond).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 NaN NaN</span>
<span class="sd"> 1 1.0 200.0</span>
<span class="sd"> 2 NaN NaN</span>
<span class="sd"> 3 NaN NaN</span>
<span class="sd"> 4 NaN NaN</span>
<span class="sd"> &gt;&gt;&gt; reset_option(&quot;compute.ops_on_diff_frames&quot;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</span><span class="p">)</span>
<span class="n">tmp_cond_col_name</span> <span class="o">=</span> <span class="s2">&quot;__tmp_cond_col_</span><span class="si">{}</span><span class="s2">__&quot;</span><span class="o">.</span><span class="n">format</span>
<span class="n">tmp_other_col_name</span> <span class="o">=</span> <span class="s2">&quot;__tmp_other_col_</span><span class="si">{}</span><span class="s2">__&quot;</span><span class="o">.</span><span class="n">format</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">tmp_cond_col_names</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">tmp_cond_col_name</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">cond</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">):</span>
<span class="n">cond</span> <span class="o">=</span> <span class="n">cond</span><span class="p">[</span>
<span class="p">[</span>
<span class="p">(</span>
<span class="n">cond</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">cond</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">else</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">)</span>
<span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name</span><span class="p">)</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">tmp_cond_col_names</span><span class="p">)</span>
<span class="p">]</span>
<span class="p">]</span>
<span class="n">psdf</span><span class="p">[</span><span class="n">tmp_cond_col_names</span><span class="p">]</span> <span class="o">=</span> <span class="n">cond</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">cond</span><span class="p">,</span> <span class="n">Series</span><span class="p">):</span>
<span class="n">cond</span> <span class="o">=</span> <span class="n">cond</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="n">cond</span> <span class="o">=</span> <span class="n">cond</span><span class="p">[</span>
<span class="p">[</span><span class="n">cond</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">alias</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">tmp_cond_col_names</span><span class="p">]</span>
<span class="p">]</span>
<span class="n">psdf</span><span class="p">[</span><span class="n">tmp_cond_col_names</span><span class="p">]</span> <span class="o">=</span> <span class="n">cond</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;type of cond must be a DataFrame or Series&quot;</span><span class="p">)</span>
<span class="n">tmp_other_col_names</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">tmp_other_col_name</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">):</span>
<span class="n">other</span> <span class="o">=</span> <span class="n">other</span><span class="p">[</span>
<span class="p">[</span>
<span class="p">(</span>
<span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">else</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">nan</span><span class="p">)</span>
<span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name</span><span class="p">)</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">tmp_other_col_names</span><span class="p">)</span>
<span class="p">]</span>
<span class="p">]</span>
<span class="n">psdf</span><span class="p">[</span><span class="n">tmp_other_col_names</span><span class="p">]</span> <span class="o">=</span> <span class="n">other</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">Series</span><span class="p">):</span>
<span class="n">other</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="n">other</span> <span class="o">=</span> <span class="n">other</span><span class="p">[</span>
<span class="p">[</span><span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">alias</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">tmp_other_col_names</span><span class="p">]</span>
<span class="p">]</span>
<span class="n">psdf</span><span class="p">[</span><span class="n">tmp_other_col_names</span><span class="p">]</span> <span class="o">=</span> <span class="n">other</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">psdf</span><span class="p">[</span><span class="n">tmp_other_col_name</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))]</span> <span class="o">=</span> <span class="n">other</span>
<span class="c1"># above logic make spark dataframe looks like below:</span>
<span class="c1"># +-----------------+---+---+------------------+-------------------+------------------+--...</span>
<span class="c1"># |__index_level_0__| A| B|__tmp_cond_col_A__|__tmp_other_col_A__|__tmp_cond_col_B__|__...</span>
<span class="c1"># +-----------------+---+---+------------------+-------------------+------------------+--...</span>
<span class="c1"># | 0| 0|100| true| 0| false| ...</span>
<span class="c1"># | 1| 1|200| false| -1| false| ...</span>
<span class="c1"># | 3| 3|400| true| -3| false| ...</span>
<span class="c1"># | 2| 2|300| false| -2| true| ...</span>
<span class="c1"># | 4| 4|500| false| -4| false| ...</span>
<span class="c1"># +-----------------+---+---+------------------+-------------------+------------------+--...</span>
<span class="n">data_spark_columns</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">data_spark_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span>
<span class="n">psdf</span><span class="p">[</span><span class="n">tmp_cond_col_name</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))]</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="p">,</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">),</span>
<span class="p">)</span>
<span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">psdf</span><span class="p">[</span><span class="n">tmp_other_col_name</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))]</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="p">)</span>
<span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</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">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span>
<span class="n">data_spark_columns</span><span class="p">,</span> <span class="n">column_labels</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span> <span class="c1"># TODO: dtypes?</span>
<span class="p">)</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.mask"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.mask.html#pyspark.pandas.DataFrame.mask">[docs]</a> <span class="k">def</span> <span class="nf">mask</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">cond</span><span class="p">:</span> <span class="n">DataFrameOrSeries</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">DataFrameOrSeries</span><span class="p">,</span> <span class="n">Any</span><span class="p">]</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">nan</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Replace values where the condition is True.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> cond : boolean DataFrame</span>
<span class="sd"> Where cond is False, keep the original value. Where True,</span>
<span class="sd"> replace with corresponding value from other.</span>
<span class="sd"> other : scalar, DataFrame</span>
<span class="sd"> Entries where cond is True are replaced with corresponding value from other.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.pandas.config import set_option, reset_option</span>
<span class="sd"> &gt;&gt;&gt; set_option(&quot;compute.ops_on_diff_frames&quot;, True)</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame({&#39;A&#39;: [0, 1, 2, 3, 4], &#39;B&#39;:[100, 200, 300, 400, 500]})</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame({&#39;A&#39;: [0, -1, -2, -3, -4], &#39;B&#39;:[-100, -200, -300, -400, -500]})</span>
<span class="sd"> &gt;&gt;&gt; df1</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 100</span>
<span class="sd"> 1 1 200</span>
<span class="sd"> 2 2 300</span>
<span class="sd"> 3 3 400</span>
<span class="sd"> 4 4 500</span>
<span class="sd"> &gt;&gt;&gt; df2</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 -100</span>
<span class="sd"> 1 -1 -200</span>
<span class="sd"> 2 -2 -300</span>
<span class="sd"> 3 -3 -400</span>
<span class="sd"> 4 -4 -500</span>
<span class="sd"> &gt;&gt;&gt; df1.mask(df1 &gt; 0).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0.0 NaN</span>
<span class="sd"> 1 NaN NaN</span>
<span class="sd"> 2 NaN NaN</span>
<span class="sd"> 3 NaN NaN</span>
<span class="sd"> 4 NaN NaN</span>
<span class="sd"> &gt;&gt;&gt; df1.mask(df1 &gt; 1, 10).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 10</span>
<span class="sd"> 1 1 10</span>
<span class="sd"> 2 10 10</span>
<span class="sd"> 3 10 10</span>
<span class="sd"> 4 10 10</span>
<span class="sd"> &gt;&gt;&gt; df1.mask(df1 &gt; 1, df1 + 100).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 200</span>
<span class="sd"> 1 1 300</span>
<span class="sd"> 2 102 400</span>
<span class="sd"> 3 103 500</span>
<span class="sd"> 4 104 600</span>
<span class="sd"> &gt;&gt;&gt; df1.mask(df1 &gt; 1, df2).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 0 -100</span>
<span class="sd"> 1 1 -200</span>
<span class="sd"> 2 -2 -300</span>
<span class="sd"> 3 -3 -400</span>
<span class="sd"> 4 -4 -500</span>
<span class="sd"> &gt;&gt;&gt; reset_option(&quot;compute.ops_on_diff_frames&quot;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">cond</span><span class="p">,</span> <span class="p">(</span><span class="n">DataFrame</span><span class="p">,</span> <span class="n">Series</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;type of cond must be a DataFrame or Series&quot;</span><span class="p">)</span>
<span class="n">cond_inversed</span> <span class="o">=</span> <span class="n">cond</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="o">~</span><span class="n">psser</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">where</span><span class="p">(</span><span class="n">cond_inversed</span><span class="p">,</span> <span class="n">other</span><span class="p">)</span></div>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">index</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Index&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;The index (row labels) Column of the DataFrame.</span>
<span class="sd"> Currently not supported when the DataFrame has no index.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Index</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.indexes.base</span> <span class="kn">import</span> <span class="n">Index</span>
<span class="k">return</span> <span class="n">Index</span><span class="o">.</span><span class="n">_new_instance</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">empty</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">bool</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns true if the current DataFrame is empty. Otherwise, returns false.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; ps.range(10).empty</span>
<span class="sd"> False</span>
<span class="sd"> &gt;&gt;&gt; ps.range(0).empty</span>
<span class="sd"> True</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame({}, index=list(&#39;abc&#39;)).empty</span>
<span class="sd"> True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="p">(</span>
<span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span>
<span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">isEmpty</span><span class="p">()</span>
<span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">style</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Styler&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Property returning a Styler object containing methods for</span>
<span class="sd"> building a styled HTML representation for the DataFrame.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; ps.range(1001).style # doctest: +SKIP</span>
<span class="sd"> &lt;pandas.io.formats.style.Styler object at ...&gt;</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">max_results</span> <span class="o">=</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.max_rows&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">max_results</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">max_results</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span> <span class="o">&gt;</span> <span class="n">max_results</span><span class="p">:</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;&#39;style&#39; property will only use top </span><span class="si">%s</span><span class="s2"> rows.&quot;</span> <span class="o">%</span> <span class="n">max_results</span><span class="p">,</span> <span class="ne">UserWarning</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">pdf</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">max_results</span><span class="p">)</span><span class="o">.</span><span class="n">style</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span><span class="o">.</span><span class="n">style</span>
<div class="viewcode-block" id="DataFrame.set_index"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.set_index.html#pyspark.pandas.DataFrame.set_index">[docs]</a> <span class="k">def</span> <span class="nf">set_index</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">keys</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]],</span>
<span class="n">drop</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">append</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Set the DataFrame index (row labels) using one or more existing columns.</span>
<span class="sd"> Set the DataFrame index (row labels) using one or more existing</span>
<span class="sd"> columns or arrays (of the correct length). The index can replace the</span>
<span class="sd"> existing index or expand on it.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> keys : label or array-like or list of labels/arrays</span>
<span class="sd"> This parameter can be either a single column key, a single array of</span>
<span class="sd"> the same length as the calling DataFrame, or a list containing an</span>
<span class="sd"> arbitrary combination of column keys and arrays. Here, &quot;array&quot;</span>
<span class="sd"> encompasses :class:`Series`, :class:`Index` and ``np.ndarray``.</span>
<span class="sd"> drop : bool, default True</span>
<span class="sd"> Delete columns to be used as the new index.</span>
<span class="sd"> append : bool, default False</span>
<span class="sd"> Whether to append columns to existing index.</span>
<span class="sd"> inplace : bool, default False</span>
<span class="sd"> Modify the DataFrame in place (do not create a new object).</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Changed row labels.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.reset_index : Opposite of set_index.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;month&#39;: [1, 4, 7, 10],</span>
<span class="sd"> ... &#39;year&#39;: [2012, 2014, 2013, 2014],</span>
<span class="sd"> ... &#39;sale&#39;: [55, 40, 84, 31]},</span>
<span class="sd"> ... columns=[&#39;month&#39;, &#39;year&#39;, &#39;sale&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> month year sale</span>
<span class="sd"> 0 1 2012 55</span>
<span class="sd"> 1 4 2014 40</span>
<span class="sd"> 2 7 2013 84</span>
<span class="sd"> 3 10 2014 31</span>
<span class="sd"> Set the index to become the &#39;month&#39; column:</span>
<span class="sd"> &gt;&gt;&gt; df.set_index(&#39;month&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> year sale</span>
<span class="sd"> month</span>
<span class="sd"> 1 2012 55</span>
<span class="sd"> 4 2014 40</span>
<span class="sd"> 7 2013 84</span>
<span class="sd"> 10 2014 31</span>
<span class="sd"> Create a MultiIndex using columns &#39;year&#39; and &#39;month&#39;:</span>
<span class="sd"> &gt;&gt;&gt; df.set_index([&#39;year&#39;, &#39;month&#39;]) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> sale</span>
<span class="sd"> year month</span>
<span class="sd"> 2012 1 55</span>
<span class="sd"> 2014 4 40</span>
<span class="sd"> 2013 7 84</span>
<span class="sd"> 2014 10 31</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="n">key_list</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span>
<span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">keys</span><span class="p">):</span>
<span class="n">key_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">cast</span><span class="p">(</span><span class="n">Label</span><span class="p">,</span> <span class="n">keys</span><span class="p">)]</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">keys</span><span class="p">):</span>
<span class="n">key_list</span> <span class="o">=</span> <span class="p">[(</span><span class="n">keys</span><span class="p">,)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">key_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">key</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">key</span><span class="p">,)</span> <span class="k">for</span> <span class="n">key</span> <span class="ow">in</span> <span class="n">keys</span><span class="p">]</span>
<span class="n">columns</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span>
<span class="k">for</span> <span class="n">key</span> <span class="ow">in</span> <span class="n">key_list</span><span class="p">:</span>
<span class="k">if</span> <span class="n">key</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">columns</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">key</span><span class="p">))</span>
<span class="k">if</span> <span class="n">drop</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">label</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span> <span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">key_list</span>
<span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">if</span> <span class="n">append</span><span class="p">:</span>
<span class="n">index_spark_columns</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span> <span class="o">+</span> <span class="p">[</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">key_list</span>
<span class="p">]</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span> <span class="o">+</span> <span class="n">key_list</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span> <span class="o">+</span> <span class="p">[</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">field_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">key_list</span>
<span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">index_spark_columns</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">key_list</span><span class="p">]</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">key_list</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">field_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">key_list</span><span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="n">index_spark_columns</span><span class="p">,</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index_names</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">],</span>
<span class="n">data_fields</span><span class="o">=</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">field_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">],</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.reset_index"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.reset_index.html#pyspark.pandas.DataFrame.reset_index">[docs]</a> <span class="k">def</span> <span class="nf">reset_index</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">level</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">]]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">drop</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">col_level</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="n">col_fill</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;&quot;</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Reset the index, or a level of it.</span>
<span class="sd"> For DataFrame with multi-level index, return new DataFrame with labeling information in</span>
<span class="sd"> the columns under the index names, defaulting to &#39;level_0&#39;, &#39;level_1&#39;, etc. if any are None.</span>
<span class="sd"> For a standard index, the index name will be used (if set), otherwise a default &#39;index&#39; or</span>
<span class="sd"> &#39;level_0&#39; (if &#39;index&#39; is already taken) will be used.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> level : int, str, tuple, or list, default None</span>
<span class="sd"> Only remove the given levels from the index. Removes all levels by</span>
<span class="sd"> default.</span>
<span class="sd"> drop : bool, default False</span>
<span class="sd"> Do not try to insert index into dataframe columns. This reset</span>
<span class="sd"> the index to the default integer index.</span>
<span class="sd"> inplace : bool, default False</span>
<span class="sd"> Modify the DataFrame in place (do not create a new object).</span>
<span class="sd"> col_level : int or str, default 0</span>
<span class="sd"> If the columns have multiple levels, determines which level the</span>
<span class="sd"> labels are inserted into. By default it is inserted into the first</span>
<span class="sd"> level.</span>
<span class="sd"> col_fill : object, default &#39;&#39;</span>
<span class="sd"> If the columns have multiple levels, determines how the other</span>
<span class="sd"> levels are named. If None then the index name is repeated.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame with the new index.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.set_index : Opposite of reset_index.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(&#39;bird&#39;, 389.0),</span>
<span class="sd"> ... (&#39;bird&#39;, 24.0),</span>
<span class="sd"> ... (&#39;mammal&#39;, 80.5),</span>
<span class="sd"> ... (&#39;mammal&#39;, np.nan)],</span>
<span class="sd"> ... index=[&#39;falcon&#39;, &#39;parrot&#39;, &#39;lion&#39;, &#39;monkey&#39;],</span>
<span class="sd"> ... columns=(&#39;class&#39;, &#39;max_speed&#39;))</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> class max_speed</span>
<span class="sd"> falcon bird 389.0</span>
<span class="sd"> parrot bird 24.0</span>
<span class="sd"> lion mammal 80.5</span>
<span class="sd"> monkey mammal NaN</span>
<span class="sd"> When we reset the index, the old index is added as a column. Unlike pandas, pandas-on-Spark</span>
<span class="sd"> does not automatically add a sequential index. The following 0, 1, 2, 3 are only</span>
<span class="sd"> there when we display the DataFrame.</span>
<span class="sd"> &gt;&gt;&gt; df.reset_index()</span>
<span class="sd"> index class max_speed</span>
<span class="sd"> 0 falcon bird 389.0</span>
<span class="sd"> 1 parrot bird 24.0</span>
<span class="sd"> 2 lion mammal 80.5</span>
<span class="sd"> 3 monkey mammal NaN</span>
<span class="sd"> We can use the `drop` parameter to avoid the old index being added as</span>
<span class="sd"> a column:</span>
<span class="sd"> &gt;&gt;&gt; df.reset_index(drop=True)</span>
<span class="sd"> class max_speed</span>
<span class="sd"> 0 bird 389.0</span>
<span class="sd"> 1 bird 24.0</span>
<span class="sd"> 2 mammal 80.5</span>
<span class="sd"> 3 mammal NaN</span>
<span class="sd"> You can also use `reset_index` with `MultiIndex`.</span>
<span class="sd"> &gt;&gt;&gt; index = pd.MultiIndex.from_tuples([(&#39;bird&#39;, &#39;falcon&#39;),</span>
<span class="sd"> ... (&#39;bird&#39;, &#39;parrot&#39;),</span>
<span class="sd"> ... (&#39;mammal&#39;, &#39;lion&#39;),</span>
<span class="sd"> ... (&#39;mammal&#39;, &#39;monkey&#39;)],</span>
<span class="sd"> ... names=[&#39;class&#39;, &#39;name&#39;])</span>
<span class="sd"> &gt;&gt;&gt; columns = pd.MultiIndex.from_tuples([(&#39;speed&#39;, &#39;max&#39;),</span>
<span class="sd"> ... (&#39;species&#39;, &#39;type&#39;)])</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(389.0, &#39;fly&#39;),</span>
<span class="sd"> ... ( 24.0, &#39;fly&#39;),</span>
<span class="sd"> ... ( 80.5, &#39;run&#39;),</span>
<span class="sd"> ... (np.nan, &#39;jump&#39;)],</span>
<span class="sd"> ... index=index,</span>
<span class="sd"> ... columns=columns)</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> speed species</span>
<span class="sd"> max type</span>
<span class="sd"> class name</span>
<span class="sd"> bird falcon 389.0 fly</span>
<span class="sd"> parrot 24.0 fly</span>
<span class="sd"> mammal lion 80.5 run</span>
<span class="sd"> monkey NaN jump</span>
<span class="sd"> If the index has multiple levels, we can reset a subset of them:</span>
<span class="sd"> &gt;&gt;&gt; df.reset_index(level=&#39;class&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> class speed species</span>
<span class="sd"> max type</span>
<span class="sd"> name</span>
<span class="sd"> falcon bird 389.0 fly</span>
<span class="sd"> parrot bird 24.0 fly</span>
<span class="sd"> lion mammal 80.5 run</span>
<span class="sd"> monkey mammal NaN jump</span>
<span class="sd"> If we are not dropping the index, by default, it is placed in the top</span>
<span class="sd"> level. We can place it in another level:</span>
<span class="sd"> &gt;&gt;&gt; df.reset_index(level=&#39;class&#39;, col_level=1) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> speed species</span>
<span class="sd"> class max type</span>
<span class="sd"> name</span>
<span class="sd"> falcon bird 389.0 fly</span>
<span class="sd"> parrot bird 24.0 fly</span>
<span class="sd"> lion mammal 80.5 run</span>
<span class="sd"> monkey mammal NaN jump</span>
<span class="sd"> When the index is inserted under another level, we can specify under</span>
<span class="sd"> which one with the parameter `col_fill`:</span>
<span class="sd"> &gt;&gt;&gt; df.reset_index(level=&#39;class&#39;, col_level=1,</span>
<span class="sd"> ... col_fill=&#39;species&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> species speed species</span>
<span class="sd"> class max type</span>
<span class="sd"> name</span>
<span class="sd"> falcon bird 389.0 fly</span>
<span class="sd"> parrot bird 24.0 fly</span>
<span class="sd"> lion mammal 80.5 run</span>
<span class="sd"> monkey mammal NaN jump</span>
<span class="sd"> If we specify a nonexistent level for `col_fill`, it is created:</span>
<span class="sd"> &gt;&gt;&gt; df.reset_index(level=&#39;class&#39;, col_level=1,</span>
<span class="sd"> ... col_fill=&#39;genus&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> genus speed species</span>
<span class="sd"> class max type</span>
<span class="sd"> name</span>
<span class="sd"> falcon bird 389.0 fly</span>
<span class="sd"> parrot bird 24.0 fly</span>
<span class="sd"> lion mammal 80.5 run</span>
<span class="sd"> monkey mammal NaN jump</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="n">multi_index</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span> <span class="o">&gt;</span> <span class="mi">1</span>
<span class="k">def</span> <span class="nf">rename</span><span class="p">(</span><span class="n">index</span><span class="p">:</span> <span class="nb">int</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Label</span><span class="p">:</span>
<span class="k">if</span> <span class="n">multi_index</span><span class="p">:</span>
<span class="k">return</span> <span class="p">(</span><span class="s2">&quot;level_</span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">index</span><span class="p">),)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="p">(</span><span class="s2">&quot;index&quot;</span><span class="p">,)</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">return</span> <span class="p">(</span><span class="s2">&quot;index&quot;</span><span class="p">,)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="p">(</span><span class="s2">&quot;level_</span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">index</span><span class="p">),)</span>
<span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">new_column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">name</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="k">else</span> <span class="n">rename</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">new_data_spark_columns</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="k">for</span> <span class="n">scol</span><span class="p">,</span> <span class="n">label</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">,</span> <span class="n">new_column_labels</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">new_data_fields</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span>
<span class="n">index_spark_columns</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">level</span><span class="p">):</span>
<span class="n">level</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">cast</span><span class="p">(</span><span class="n">Sequence</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">]],</span> <span class="n">level</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">level</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="ow">or</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">level</span><span class="p">):</span>
<span class="n">level_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">cast</span><span class="p">(</span><span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Label</span><span class="p">],</span> <span class="n">level</span><span class="p">)]</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">level</span><span class="p">):</span>
<span class="n">level_list</span> <span class="o">=</span> <span class="p">[(</span><span class="n">level</span><span class="p">,)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">level_list</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">lvl</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">lvl</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="ow">or</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">lvl</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">lvl</span><span class="p">,)</span>
<span class="k">for</span> <span class="n">lvl</span> <span class="ow">in</span> <span class="n">level</span>
<span class="p">]</span>
<span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">lvl</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="k">for</span> <span class="n">lvl</span> <span class="ow">in</span> <span class="n">level_list</span><span class="p">):</span>
<span class="n">int_level_list</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">List</span><span class="p">[</span><span class="nb">int</span><span class="p">],</span> <span class="n">level_list</span><span class="p">)</span>
<span class="k">for</span> <span class="n">lev</span> <span class="ow">in</span> <span class="n">int_level_list</span><span class="p">:</span>
<span class="k">if</span> <span class="n">lev</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">IndexError</span><span class="p">(</span>
<span class="s2">&quot;Too many levels: Index has only </span><span class="si">{}</span><span class="s2"> level, not </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">,</span> <span class="n">lev</span> <span class="o">+</span> <span class="mi">1</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">idx</span> <span class="o">=</span> <span class="n">int_level_list</span>
<span class="k">elif</span> <span class="nb">all</span><span class="p">(</span><span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">lev</span><span class="p">)</span> <span class="k">for</span> <span class="n">lev</span> <span class="ow">in</span> <span class="n">level_list</span><span class="p">):</span>
<span class="n">idx</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">cast</span><span class="p">(</span><span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">],</span> <span class="n">level_list</span><span class="p">):</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">i</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">idx</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">ValueError</span><span class="p">:</span>
<span class="k">if</span> <span class="n">multi_index</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="s2">&quot;Level unknown not found&quot;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span>
<span class="s2">&quot;Level unknown must be same as name (</span><span class="si">{}</span><span class="s2">)&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">name_like_string</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Level should be all int or all string.&quot;</span><span class="p">)</span>
<span class="n">idx</span><span class="o">.</span><span class="n">sort</span><span class="p">()</span>
<span class="n">new_column_labels</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">new_data_spark_columns</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">new_data_fields</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">index_spark_columns</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">idx</span><span class="p">[::</span><span class="o">-</span><span class="mi">1</span><span class="p">]:</span>
<span class="n">name</span> <span class="o">=</span> <span class="n">index_names</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="n">new_column_labels</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">name</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="k">else</span> <span class="n">rename</span><span class="p">(</span><span class="n">i</span><span class="p">))</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">index_spark_columns</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="n">new_data_spark_columns</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">name</span><span class="p">)))</span>
<span class="n">new_data_fields</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">index_fields</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="n">i</span><span class="p">)</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">name</span><span class="p">)))</span>
<span class="k">if</span> <span class="n">drop</span><span class="p">:</span>
<span class="n">new_data_spark_columns</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">new_column_labels</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">new_data_fields</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">new_column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;cannot insert </span><span class="si">{}</span><span class="s2">, already exists&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">)))</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">column_depth</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span>
<span class="k">if</span> <span class="n">col_level</span> <span class="o">&gt;=</span> <span class="n">column_depth</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">IndexError</span><span class="p">(</span>
<span class="s2">&quot;Too many levels: Index has only </span><span class="si">{}</span><span class="s2"> levels, not </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">column_depth</span><span class="p">,</span> <span class="n">col_level</span> <span class="o">+</span> <span class="mi">1</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="n">col_level</span> <span class="o">+</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">&gt;</span> <span class="n">column_depth</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">new_column_labels</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Item must have length equal to number of levels.&quot;</span><span class="p">)</span>
<span class="n">new_column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="nb">tuple</span><span class="p">(</span>
<span class="p">([</span><span class="n">col_fill</span><span class="p">]</span> <span class="o">*</span> <span class="n">col_level</span><span class="p">)</span>
<span class="o">+</span> <span class="nb">list</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="o">+</span> <span class="p">([</span><span class="n">col_fill</span><span class="p">]</span> <span class="o">*</span> <span class="p">(</span><span class="n">column_depth</span> <span class="o">-</span> <span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">+</span> <span class="n">col_level</span><span class="p">)))</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">new_column_labels</span>
<span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="n">index_spark_columns</span><span class="p">,</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index_names</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">new_column_labels</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="n">new_data_spark_columns</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">,</span>
<span class="n">data_fields</span><span class="o">=</span><span class="n">new_data_fields</span> <span class="o">+</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.isnull"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.isnull.html#pyspark.pandas.DataFrame.isnull">[docs]</a> <span class="k">def</span> <span class="nf">isnull</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Detects missing values for items in the current Dataframe.</span>
<span class="sd"> Return a boolean same-sized Dataframe indicating if the values are NA.</span>
<span class="sd"> NA values, such as None or numpy.NaN, gets mapped to True values.</span>
<span class="sd"> Everything else gets mapped to False values.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.notnull</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(.2, .3), (.0, None), (.6, None), (.2, .1)])</span>
<span class="sd"> &gt;&gt;&gt; df.isnull()</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 False False</span>
<span class="sd"> 1 False True</span>
<span class="sd"> 2 False True</span>
<span class="sd"> 3 False False</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[None, &#39;bee&#39;, None], [&#39;dog&#39;, None, &#39;fly&#39;]])</span>
<span class="sd"> &gt;&gt;&gt; df.isnull()</span>
<span class="sd"> 0 1 2</span>
<span class="sd"> 0 True False True</span>
<span class="sd"> 1 False True False</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">isnull</span><span class="p">())</span></div>
<span class="n">isna</span> <span class="o">=</span> <span class="n">isnull</span>
<div class="viewcode-block" id="DataFrame.notnull"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.notnull.html#pyspark.pandas.DataFrame.notnull">[docs]</a> <span class="k">def</span> <span class="nf">notnull</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Detects non-missing values for items in the current Dataframe.</span>
<span class="sd"> This function takes a dataframe and indicates whether it&#39;s</span>
<span class="sd"> values are valid (not missing, which is ``NaN`` in numeric</span>
<span class="sd"> datatypes, ``None`` or ``NaN`` in objects and ``NaT`` in datetimelike).</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.isnull</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(.2, .3), (.0, None), (.6, None), (.2, .1)])</span>
<span class="sd"> &gt;&gt;&gt; df.notnull()</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 True True</span>
<span class="sd"> 1 True False</span>
<span class="sd"> 2 True False</span>
<span class="sd"> 3 True True</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[&#39;ant&#39;, &#39;bee&#39;, &#39;cat&#39;], [&#39;dog&#39;, None, &#39;fly&#39;]])</span>
<span class="sd"> &gt;&gt;&gt; df.notnull()</span>
<span class="sd"> 0 1 2</span>
<span class="sd"> 0 True True True</span>
<span class="sd"> 1 True False True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">notnull</span><span class="p">())</span></div>
<span class="n">notna</span> <span class="o">=</span> <span class="n">notnull</span>
<div class="viewcode-block" id="DataFrame.insert"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.insert.html#pyspark.pandas.DataFrame.insert">[docs]</a> <span class="k">def</span> <span class="nf">insert</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">loc</span><span class="p">:</span> <span class="nb">int</span><span class="p">,</span>
<span class="n">column</span><span class="p">:</span> <span class="n">Name</span><span class="p">,</span>
<span class="n">value</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Scalar</span><span class="p">,</span> <span class="s2">&quot;Series&quot;</span><span class="p">,</span> <span class="n">Iterable</span><span class="p">],</span>
<span class="n">allow_duplicates</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</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;</span>
<span class="sd"> Insert column into DataFrame at specified location.</span>
<span class="sd"> Raises a ValueError if `column` is already contained in the DataFrame,</span>
<span class="sd"> unless `allow_duplicates` is set to True.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> loc : int</span>
<span class="sd"> Insertion index. Must verify 0 &lt;= loc &lt;= len(columns).</span>
<span class="sd"> column : str, number, or hashable object</span>
<span class="sd"> Label of the inserted column.</span>
<span class="sd"> value : int, Series, or array-like</span>
<span class="sd"> allow_duplicates : bool, optional</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame([1, 2, 3])</span>
<span class="sd"> &gt;&gt;&gt; psdf.sort_index()</span>
<span class="sd"> 0</span>
<span class="sd"> 0 1</span>
<span class="sd"> 1 2</span>
<span class="sd"> 2 3</span>
<span class="sd"> &gt;&gt;&gt; psdf.insert(0, &#39;x&#39;, 4)</span>
<span class="sd"> &gt;&gt;&gt; psdf.sort_index()</span>
<span class="sd"> x 0</span>
<span class="sd"> 0 4 1</span>
<span class="sd"> 1 4 2</span>
<span class="sd"> 2 4 3</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.pandas.config import set_option, reset_option</span>
<span class="sd"> &gt;&gt;&gt; set_option(&quot;compute.ops_on_diff_frames&quot;, True)</span>
<span class="sd"> &gt;&gt;&gt; psdf.insert(1, &#39;y&#39;, [5, 6, 7])</span>
<span class="sd"> &gt;&gt;&gt; psdf.sort_index()</span>
<span class="sd"> x y 0</span>
<span class="sd"> 0 4 5 1</span>
<span class="sd"> 1 4 6 2</span>
<span class="sd"> 2 4 7 3</span>
<span class="sd"> &gt;&gt;&gt; psdf.insert(2, &#39;z&#39;, ps.Series([8, 9, 10]))</span>
<span class="sd"> &gt;&gt;&gt; psdf.sort_index()</span>
<span class="sd"> x y z 0</span>
<span class="sd"> 0 4 5 8 1</span>
<span class="sd"> 1 4 6 9 2</span>
<span class="sd"> 2 4 7 10 3</span>
<span class="sd"> &gt;&gt;&gt; reset_option(&quot;compute.ops_on_diff_frames&quot;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">loc</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;loc must be int&quot;</span><span class="p">)</span>
<span class="k">assert</span> <span class="mi">0</span> <span class="o">&lt;=</span> <span class="n">loc</span> <span class="o">&lt;=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span>
<span class="k">assert</span> <span class="n">allow_duplicates</span> <span class="ow">is</span> <span class="kc">False</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">column</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s1">&#39;&quot;column&quot; should be a scalar value or tuple that contains scalar values&#39;</span>
<span class="p">)</span>
<span class="c1"># TODO(SPARK-37723): Support tuple for non-MultiIndex column name.</span>
<span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">column</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">column</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">levels</span><span class="p">):</span> <span class="c1"># type: ignore[attr-defined]</span>
<span class="c1"># To be consistent with pandas</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;&quot;column&quot; must have length equal to number of column levels.&#39;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span>
<span class="s2">&quot;Assigning column name as tuple is only supported for MultiIndex columns &quot;</span>
<span class="s2">&quot;for now.&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">column</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;cannot insert </span><span class="si">%s</span><span class="s2">, already exists&quot;</span> <span class="o">%</span> <span class="nb">str</span><span class="p">(</span><span class="n">column</span><span class="p">))</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">psdf</span><span class="p">[</span><span class="n">column</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span>
<span class="n">columns</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span><span class="o">.</span><span class="n">insert</span><span class="p">(</span><span class="n">loc</span><span class="p">,</span> <span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">])</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">psdf</span><span class="p">[</span><span class="n">columns</span><span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46156): add frep and axis parameter</span>
<div class="viewcode-block" id="DataFrame.shift"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.shift.html#pyspark.pandas.DataFrame.shift">[docs]</a> <span class="k">def</span> <span class="nf">shift</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">periods</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="mi">1</span><span class="p">,</span> <span class="n">fill_value</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Any</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="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Shift DataFrame by desired number of periods.</span>
<span class="sd"> .. note:: the current implementation of shift uses Spark&#39;s Window without</span>
<span class="sd"> specifying partition specification. This leads to moving all data into</span>
<span class="sd"> a single partition in a single machine and could cause serious</span>
<span class="sd"> performance degradation. Avoid this method with very large datasets.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> periods : int</span>
<span class="sd"> Number of periods to shift. Can be positive or negative.</span>
<span class="sd"> fill_value : object, optional</span>
<span class="sd"> The scalar value to use for newly introduced missing values.</span>
<span class="sd"> The default depends on the dtype of self. For numeric data, np.nan is used.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Copy of input DataFrame, shifted.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;Col1&#39;: [10, 20, 15, 30, 45],</span>
<span class="sd"> ... &#39;Col2&#39;: [13, 23, 18, 33, 48],</span>
<span class="sd"> ... &#39;Col3&#39;: [17, 27, 22, 37, 52]},</span>
<span class="sd"> ... columns=[&#39;Col1&#39;, &#39;Col2&#39;, &#39;Col3&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.shift(periods=3)</span>
<span class="sd"> Col1 Col2 Col3</span>
<span class="sd"> 0 NaN NaN NaN</span>
<span class="sd"> 1 NaN NaN NaN</span>
<span class="sd"> 2 NaN NaN NaN</span>
<span class="sd"> 3 10.0 13.0 17.0</span>
<span class="sd"> 4 20.0 23.0 27.0</span>
<span class="sd"> &gt;&gt;&gt; df.shift(periods=3, fill_value=0)</span>
<span class="sd"> Col1 Col2 Col3</span>
<span class="sd"> 0 0 0 0</span>
<span class="sd"> 1 0 0 0</span>
<span class="sd"> 2 0 0 0</span>
<span class="sd"> 3 10 13 17</span>
<span class="sd"> 4 20 23 27</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">_shift</span><span class="p">(</span><span class="n">periods</span><span class="p">,</span> <span class="n">fill_value</span><span class="p">),</span> <span class="n">should_resolve</span><span class="o">=</span><span class="kc">True</span>
<span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46161): axis should support 1 or &#39;columns&#39; either at this moment</span>
<div class="viewcode-block" id="DataFrame.diff"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.diff.html#pyspark.pandas.DataFrame.diff">[docs]</a> <span class="k">def</span> <span class="nf">diff</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">periods</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="mi">1</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> First discrete difference of element.</span>
<span class="sd"> Calculates the difference of a DataFrame element compared with another element in the</span>
<span class="sd"> DataFrame (default is the element in the same column of the previous row).</span>
<span class="sd"> .. note:: the current implementation of diff uses Spark&#39;s Window without</span>
<span class="sd"> specifying partition specification. This leads to moving all data into</span>
<span class="sd"> a single partition in a single machine and could cause serious</span>
<span class="sd"> performance degradation. Avoid this method with very large datasets.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> periods : int, default 1</span>
<span class="sd"> Periods to shift for calculating difference, accepts negative values.</span>
<span class="sd"> axis : int, default 0 or &#39;index&#39;</span>
<span class="sd"> Can only be set to 0 now.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> diffed : DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 4, 5, 6],</span>
<span class="sd"> ... &#39;b&#39;: [1, 1, 2, 3, 5, 8],</span>
<span class="sd"> ... &#39;c&#39;: [1, 4, 9, 16, 25, 36]}, columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> a b c</span>
<span class="sd"> 0 1 1 1</span>
<span class="sd"> 1 2 1 4</span>
<span class="sd"> 2 3 2 9</span>
<span class="sd"> 3 4 3 16</span>
<span class="sd"> 4 5 5 25</span>
<span class="sd"> 5 6 8 36</span>
<span class="sd"> &gt;&gt;&gt; df.diff()</span>
<span class="sd"> a b c</span>
<span class="sd"> 0 NaN NaN NaN</span>
<span class="sd"> 1 1.0 0.0 3.0</span>
<span class="sd"> 2 1.0 1.0 5.0</span>
<span class="sd"> 3 1.0 1.0 7.0</span>
<span class="sd"> 4 1.0 2.0 9.0</span>
<span class="sd"> 5 1.0 3.0 11.0</span>
<span class="sd"> Difference with previous column</span>
<span class="sd"> &gt;&gt;&gt; df.diff(periods=3)</span>
<span class="sd"> a b c</span>
<span class="sd"> 0 NaN NaN NaN</span>
<span class="sd"> 1 NaN NaN NaN</span>
<span class="sd"> 2 NaN NaN NaN</span>
<span class="sd"> 3 3.0 2.0 15.0</span>
<span class="sd"> 4 3.0 4.0 21.0</span>
<span class="sd"> 5 3.0 6.0 27.0</span>
<span class="sd"> Difference with following row</span>
<span class="sd"> &gt;&gt;&gt; df.diff(periods=-1)</span>
<span class="sd"> a b c</span>
<span class="sd"> 0 -1.0 0.0 -3.0</span>
<span class="sd"> 1 -1.0 -1.0 -5.0</span>
<span class="sd"> 2 -1.0 -1.0 -7.0</span>
<span class="sd"> 3 -1.0 -2.0 -9.0</span>
<span class="sd"> 4 -1.0 -3.0 -11.0</span>
<span class="sd"> 5 NaN NaN NaN</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">_diff</span><span class="p">(</span><span class="n">periods</span><span class="p">),</span> <span class="n">should_resolve</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46162): axis should support 1 or &#39;columns&#39; either at this moment</span>
<div class="viewcode-block" id="DataFrame.nunique"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.nunique.html#pyspark.pandas.DataFrame.nunique">[docs]</a> <span class="k">def</span> <span class="nf">nunique</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="n">dropna</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">approx</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">rsd</span><span class="p">:</span> <span class="nb">float</span> <span class="o">=</span> <span class="mf">0.05</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return number of unique elements in the object.</span>
<span class="sd"> Excludes NA values by default.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> axis : int, default 0 or &#39;index&#39;</span>
<span class="sd"> Can only be set to 0 now.</span>
<span class="sd"> dropna : bool, default True</span>
<span class="sd"> Don’t include NaN in the count.</span>
<span class="sd"> approx: bool, default False</span>
<span class="sd"> If False, will use the exact algorithm and return the exact number of unique.</span>
<span class="sd"> If True, it uses the HyperLogLog approximate algorithm, which is significantly faster</span>
<span class="sd"> for large amounts of data.</span>
<span class="sd"> Note: This parameter is specific to pandas-on-Spark and is not found in pandas.</span>
<span class="sd"> rsd: float, default 0.05</span>
<span class="sd"> Maximum estimation error allowed in the HyperLogLog algorithm.</span>
<span class="sd"> Note: Just like ``approx`` this parameter is specific to pandas-on-Spark.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> The number of unique values per column as a pandas-on-Spark Series.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [1, 2, 3], &#39;B&#39;: [np.nan, 3, np.nan]})</span>
<span class="sd"> &gt;&gt;&gt; df.nunique()</span>
<span class="sd"> A 3</span>
<span class="sd"> B 1</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> &gt;&gt;&gt; df.nunique(dropna=False)</span>
<span class="sd"> A 3</span>
<span class="sd"> B 2</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> On big data, we recommend using the approximate algorithm to speed up this function.</span>
<span class="sd"> The result will be very close to the exact unique count.</span>
<span class="sd"> &gt;&gt;&gt; df.nunique(approx=True)</span>
<span class="sd"> A 3</span>
<span class="sd"> B 1</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">first_series</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">StringType</span><span class="p">())</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">)]</span>
<span class="o">+</span> <span class="p">[</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">_nunique</span><span class="p">(</span><span class="n">dropna</span><span class="p">,</span> <span class="n">approx</span><span class="p">,</span> <span class="n">rsd</span><span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">]</span>
<span class="p">)</span>
<span class="c1"># The data is expected to be small so it&#39;s fine to transpose/use the default index.</span>
<span class="k">with</span> <span class="n">ps</span><span class="o">.</span><span class="n">option_context</span><span class="p">(</span><span class="s2">&quot;compute.max_rows&quot;</span><span class="p">,</span> <span class="mi">1</span><span class="p">):</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">)],</span>
<span class="n">index_names</span><span class="o">=</span><span class="p">[</span><span class="kc">None</span><span class="p">],</span>
<span class="n">index_fields</span><span class="o">=</span><span class="p">[</span><span class="kc">None</span><span class="p">],</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="p">],</span>
<span class="n">data_fields</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span><span class="o">.</span><span class="n">transpose</span><span class="p">())</span></div>
<div class="viewcode-block" id="DataFrame.round"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.round.html#pyspark.pandas.DataFrame.round">[docs]</a> <span class="k">def</span> <span class="nf">round</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">decimals</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="nb">int</span><span class="p">],</span> <span class="s2">&quot;Series&quot;</span><span class="p">]</span> <span class="o">=</span> <span class="mi">0</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Round a DataFrame to a variable number of decimal places.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> decimals : int, dict, Series</span>
<span class="sd"> Number of decimal places to round each column to. If an int is</span>
<span class="sd"> given, round each column to the same number of places.</span>
<span class="sd"> Otherwise dict and Series round to variable numbers of places.</span>
<span class="sd"> Column names should be in the keys if `decimals` is a</span>
<span class="sd"> dict-like, or in the index if `decimals` is a Series. Any</span>
<span class="sd"> columns not included in `decimals` will be left as is. Elements</span>
<span class="sd"> of `decimals` which are not columns of the input will be</span>
<span class="sd"> ignored.</span>
<span class="sd"> .. note:: If `decimals` is a Series, it is expected to be small,</span>
<span class="sd"> as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.round</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;:[0.028208, 0.038683, 0.877076],</span>
<span class="sd"> ... &#39;B&#39;:[0.992815, 0.645646, 0.149370],</span>
<span class="sd"> ... &#39;C&#39;:[0.173891, 0.577595, 0.491027]},</span>
<span class="sd"> ... columns=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;],</span>
<span class="sd"> ... index=[&#39;first&#39;, &#39;second&#39;, &#39;third&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C</span>
<span class="sd"> first 0.028208 0.992815 0.173891</span>
<span class="sd"> second 0.038683 0.645646 0.577595</span>
<span class="sd"> third 0.877076 0.149370 0.491027</span>
<span class="sd"> &gt;&gt;&gt; df.round(2)</span>
<span class="sd"> A B C</span>
<span class="sd"> first 0.03 0.99 0.17</span>
<span class="sd"> second 0.04 0.65 0.58</span>
<span class="sd"> third 0.88 0.15 0.49</span>
<span class="sd"> &gt;&gt;&gt; df.round({&#39;A&#39;: 1, &#39;C&#39;: 2})</span>
<span class="sd"> A B C</span>
<span class="sd"> first 0.0 0.992815 0.17</span>
<span class="sd"> second 0.0 0.645646 0.58</span>
<span class="sd"> third 0.9 0.149370 0.49</span>
<span class="sd"> &gt;&gt;&gt; decimals = ps.Series([1, 0, 2], index=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.round(decimals)</span>
<span class="sd"> A B C</span>
<span class="sd"> first 0.0 1.0 0.17</span>
<span class="sd"> second 0.0 1.0 0.58</span>
<span class="sd"> third 0.9 0.0 0.49</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">decimals</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="n">decimals_dict</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">k</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">)</span> <span class="k">else</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">decimals</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">decimals</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="n">decimals_dict</span> <span class="o">=</span> <span class="p">{</span><span class="n">k</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">k</span><span class="p">)</span> <span class="k">else</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">decimals</span><span class="o">.</span><span class="n">items</span><span class="p">()}</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">decimals</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="n">decimals_dict</span> <span class="o">=</span> <span class="p">{</span><span class="n">k</span><span class="p">:</span> <span class="n">decimals</span> <span class="k">for</span> <span class="n">k</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">}</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;decimals must be an integer, a dict-like or a Series&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">op</span><span class="p">(</span><span class="n">psser</span><span class="p">:</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Union</span><span class="p">[</span><span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">,</span> <span class="n">PySparkColumn</span><span class="p">]:</span>
<span class="n">label</span> <span class="o">=</span> <span class="n">psser</span><span class="o">.</span><span class="n">_column_label</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">decimals_dict</span><span class="p">:</span>
<span class="k">return</span> <span class="n">F</span><span class="o">.</span><span class="n">round</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="p">,</span> <span class="n">decimals_dict</span><span class="p">[</span><span class="n">label</span><span class="p">])</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psser</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="n">op</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_mark_duplicates</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">subset</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">keep</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">bool</span><span class="p">,</span> <span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="s2">&quot;first&quot;</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Tuple</span><span class="p">[</span><span class="n">PySparkDataFrame</span><span class="p">,</span> <span class="nb">str</span><span class="p">]:</span>
<span class="k">if</span> <span class="n">subset</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">subset_list</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">subset</span><span class="p">):</span>
<span class="n">subset_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">cast</span><span class="p">(</span><span class="n">Label</span><span class="p">,</span> <span class="n">subset</span><span class="p">)]</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">subset</span><span class="p">):</span>
<span class="n">subset_list</span> <span class="o">=</span> <span class="p">[(</span><span class="n">subset</span><span class="p">,)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">subset_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">sub</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">sub</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">sub</span><span class="p">,)</span> <span class="k">for</span> <span class="n">sub</span> <span class="ow">in</span> <span class="n">subset</span><span class="p">]</span>
<span class="n">diff</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">subset_list</span><span class="p">)</span><span class="o">.</span><span class="n">difference</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">diff</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="s2">&quot;, &quot;</span><span class="o">.</span><span class="n">join</span><span class="p">([</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">d</span><span class="p">)</span> <span class="k">for</span> <span class="n">d</span> <span class="ow">in</span> <span class="n">diff</span><span class="p">]))</span>
<span class="n">group_cols</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">subset_list</span><span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="n">column</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;__duplicated__&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s2">&quot;first&quot;</span> <span class="ow">or</span> <span class="n">keep</span> <span class="o">==</span> <span class="s2">&quot;last&quot;</span><span class="p">:</span>
<span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s2">&quot;first&quot;</span><span class="p">:</span>
<span class="n">ord_func</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">asc</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">ord_func</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">desc</span>
<span class="n">window</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">Window</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="o">*</span><span class="n">group_cols</span><span class="p">)</span>
<span class="o">.</span><span class="n">orderBy</span><span class="p">(</span><span class="n">ord_func</span><span class="p">(</span><span class="n">NATURAL_ORDER_COLUMN_NAME</span><span class="p">))</span>
<span class="o">.</span><span class="n">rowsBetween</span><span class="p">(</span><span class="n">Window</span><span class="o">.</span><span class="n">unboundedPreceding</span><span class="p">,</span> <span class="n">Window</span><span class="o">.</span><span class="n">currentRow</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="n">column</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">row_number</span><span class="p">()</span><span class="o">.</span><span class="n">over</span><span class="p">(</span><span class="n">window</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">)</span>
<span class="k">elif</span> <span class="ow">not</span> <span class="n">keep</span><span class="p">:</span>
<span class="n">window</span> <span class="o">=</span> <span class="n">Window</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="o">*</span><span class="n">group_cols</span><span class="p">)</span><span class="o">.</span><span class="n">rowsBetween</span><span class="p">(</span>
<span class="n">Window</span><span class="o">.</span><span class="n">unboundedPreceding</span><span class="p">,</span> <span class="n">Window</span><span class="o">.</span><span class="n">unboundedFollowing</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="n">column</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">count</span><span class="p">(</span><span class="s2">&quot;*&quot;</span><span class="p">)</span><span class="o">.</span><span class="n">over</span><span class="p">(</span><span class="n">window</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;&#39;keep&#39; only supports &#39;first&#39;, &#39;last&#39; and False&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">sdf</span><span class="p">,</span> <span class="n">column</span>
<div class="viewcode-block" id="DataFrame.duplicated"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.duplicated.html#pyspark.pandas.DataFrame.duplicated">[docs]</a> <span class="k">def</span> <span class="nf">duplicated</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">subset</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">keep</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">bool</span><span class="p">,</span> <span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="s2">&quot;first&quot;</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return boolean Series denoting duplicate rows, optionally only considering certain columns.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> subset : column label or sequence of labels, optional</span>
<span class="sd"> Only consider certain columns for identifying duplicates,</span>
<span class="sd"> default use all of the columns</span>
<span class="sd"> keep : {&#39;first&#39;, &#39;last&#39;, False}, default &#39;first&#39;</span>
<span class="sd"> - ``first`` : Mark duplicates as ``True`` except for the first occurrence.</span>
<span class="sd"> - ``last`` : Mark duplicates as ``True`` except for the last occurrence.</span>
<span class="sd"> - False : Mark all duplicates as ``True``.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> duplicated : Series</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 1, 1, 3], &#39;b&#39;: [1, 1, 1, 4], &#39;c&#39;: [1, 1, 1, 5]},</span>
<span class="sd"> ... columns = [&#39;a&#39;, &#39;b&#39;, &#39;c&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> a b c</span>
<span class="sd"> 0 1 1 1</span>
<span class="sd"> 1 1 1 1</span>
<span class="sd"> 2 1 1 1</span>
<span class="sd"> 3 3 4 5</span>
<span class="sd"> &gt;&gt;&gt; df.duplicated().sort_index()</span>
<span class="sd"> 0 False</span>
<span class="sd"> 1 True</span>
<span class="sd"> 2 True</span>
<span class="sd"> 3 False</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> Mark duplicates as ``True`` except for the last occurrence.</span>
<span class="sd"> &gt;&gt;&gt; df.duplicated(keep=&#39;last&#39;).sort_index()</span>
<span class="sd"> 0 True</span>
<span class="sd"> 1 True</span>
<span class="sd"> 2 False</span>
<span class="sd"> 3 False</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> Mark all duplicates as ``True``.</span>
<span class="sd"> &gt;&gt;&gt; df.duplicated(keep=False).sort_index()</span>
<span class="sd"> 0 True</span>
<span class="sd"> 1 True</span>
<span class="sd"> 2 True</span>
<span class="sd"> 3 False</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">first_series</span>
<span class="n">sdf</span><span class="p">,</span> <span class="n">column</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_mark_duplicates</span><span class="p">(</span><span class="n">subset</span><span class="p">,</span> <span class="n">keep</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="o">+</span> <span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">column</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">SPARK_DEFAULT_SERIES_NAME</span><span class="p">)]</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span>
<span class="n">DataFrame</span><span class="p">(</span>
<span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="p">[</span><span class="kc">None</span><span class="p">],</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">SPARK_DEFAULT_SERIES_NAME</span><span class="p">)],</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="p">)</span></div>
<span class="c1"># TODO: support other as DataFrame or array-like</span>
<div class="viewcode-block" id="DataFrame.dot"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.dot.html#pyspark.pandas.DataFrame.dot">[docs]</a> <span class="k">def</span> <span class="nf">dot</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="s2">&quot;Series&quot;</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute the matrix multiplication between the DataFrame and others.</span>
<span class="sd"> This method computes the matrix product between the DataFrame and the</span>
<span class="sd"> values of an other Series</span>
<span class="sd"> It can also be called using ``self @ other`` in Python &gt;= 3.5.</span>
<span class="sd"> .. note:: This method is based on an expensive operation due to the nature</span>
<span class="sd"> of big data. Internally it needs to generate each row for each value, and</span>
<span class="sd"> then group twice - it is a huge operation. To prevent misuse, this method</span>
<span class="sd"> has the &#39;compute.max_rows&#39; default limit of input length and raises a ValueError.</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.pandas.config import option_context</span>
<span class="sd"> &gt;&gt;&gt; with option_context(</span>
<span class="sd"> ... &#39;compute.max_rows&#39;, 1000, &quot;compute.ops_on_diff_frames&quot;, True</span>
<span class="sd"> ... ): # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> ... psdf = ps.DataFrame({&#39;a&#39;: range(1001)})</span>
<span class="sd"> ... psser = ps.Series([2], index=[&#39;a&#39;])</span>
<span class="sd"> ... psdf.dot(psser)</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: Current DataFrame&#39;s length exceeds the given limit of 1000 rows.</span>
<span class="sd"> Please set &#39;compute.max_rows&#39; by using &#39;pyspark.pandas.config.set_option&#39;</span>
<span class="sd"> to retrieve more than 1000 rows. Note that, before changing the</span>
<span class="sd"> &#39;compute.max_rows&#39;, this operation is considerably expensive.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> other : Series</span>
<span class="sd"> The other object to compute the matrix product with.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series</span>
<span class="sd"> Return the matrix product between self and other as a Series.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.dot: Similar method for Series.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> The dimensions of DataFrame and other must be compatible to</span>
<span class="sd"> compute the matrix multiplication. In addition, the column names of</span>
<span class="sd"> DataFrame and the index of other must contain the same values, as they</span>
<span class="sd"> will be aligned prior to the multiplication.</span>
<span class="sd"> The dot method for Series computes the inner product, instead of the</span>
<span class="sd"> matrix product here.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.pandas.config import set_option, reset_option</span>
<span class="sd"> &gt;&gt;&gt; set_option(&quot;compute.ops_on_diff_frames&quot;, True)</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame([[0, 1, -2, -1], [1, 1, 1, 1]])</span>
<span class="sd"> &gt;&gt;&gt; psser = ps.Series([1, 1, 2, 1])</span>
<span class="sd"> &gt;&gt;&gt; psdf.dot(psser)</span>
<span class="sd"> 0 -4</span>
<span class="sd"> 1 5</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> Note how shuffling of the objects does not change the result.</span>
<span class="sd"> &gt;&gt;&gt; psser2 = psser.reindex([1, 0, 2, 3])</span>
<span class="sd"> &gt;&gt;&gt; psdf.dot(psser2)</span>
<span class="sd"> 0 -4</span>
<span class="sd"> 1 5</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> &gt;&gt;&gt; psdf @ psser2</span>
<span class="sd"> 0 -4</span>
<span class="sd"> 1 5</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> &gt;&gt;&gt; reset_option(&quot;compute.ops_on_diff_frames&quot;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Unsupported type </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span><span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">dot</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">transpose</span><span class="p">()))</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span></div>
<span class="k">def</span> <span class="fm">__matmul__</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="s2">&quot;Series&quot;</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Matrix multiplication using binary `@` operator in Python&gt;=3.5.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">dot</span><span class="p">(</span><span class="n">other</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.to_table"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_table.html#pyspark.pandas.DataFrame.to_table">[docs]</a> <span class="k">def</span> <span class="nf">to_table</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="nb">str</span><span class="p">,</span>
<span class="nb">format</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="n">mode</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;w&quot;</span><span class="p">,</span>
<span class="n">partition_cols</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index_col</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="o">**</span><span class="n">options</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">index_col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="s2">&quot;If `index_col` is not specified for `to_table`, &quot;</span>
<span class="s2">&quot;the existing index is lost when converting to table.&quot;</span>
<span class="p">)</span>
<span class="n">mode</span> <span class="o">=</span> <span class="n">validate_mode</span><span class="p">(</span><span class="n">mode</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">to_table</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="nb">format</span><span class="p">,</span> <span class="n">mode</span><span class="p">,</span> <span class="n">partition_cols</span><span class="p">,</span> <span class="n">index_col</span><span class="p">,</span> <span class="o">**</span><span class="n">options</span><span class="p">)</span></div>
<span class="n">to_table</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">SparkFrameMethods</span><span class="o">.</span><span class="n">to_table</span><span class="o">.</span><span class="vm">__doc__</span>
<div class="viewcode-block" id="DataFrame.to_delta"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_delta.html#pyspark.pandas.DataFrame.to_delta">[docs]</a> <span class="k">def</span> <span class="nf">to_delta</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">path</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span>
<span class="n">mode</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;w&quot;</span><span class="p">,</span>
<span class="n">partition_cols</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index_col</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="o">**</span><span class="n">options</span><span class="p">:</span> <span class="s2">&quot;OptionalPrimitiveType&quot;</span><span class="p">,</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;</span>
<span class="sd"> Write the DataFrame out as a Delta Lake table.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> path : str, required</span>
<span class="sd"> Path to write to.</span>
<span class="sd"> mode : str</span>
<span class="sd"> Python write mode, default &#39;w&#39;.</span>
<span class="sd"> .. note:: mode can accept the strings for Spark writing mode.</span>
<span class="sd"> Such as &#39;append&#39;, &#39;overwrite&#39;, &#39;ignore&#39;, &#39;error&#39;, &#39;errorifexists&#39;.</span>
<span class="sd"> - &#39;append&#39; (equivalent to &#39;a&#39;): Append the new data to existing data.</span>
<span class="sd"> - &#39;overwrite&#39; (equivalent to &#39;w&#39;): Overwrite existing data.</span>
<span class="sd"> - &#39;ignore&#39;: Silently ignore this operation if data already exists.</span>
<span class="sd"> - &#39;error&#39; or &#39;errorifexists&#39;: Throw an exception if data already exists.</span>
<span class="sd"> partition_cols : str or list of str, optional, default None</span>
<span class="sd"> Names of partitioning columns</span>
<span class="sd"> index_col: str or list of str, optional, default: None</span>
<span class="sd"> Column names to be used in Spark to represent pandas-on-Spark&#39;s index. The index name</span>
<span class="sd"> in pandas-on-Spark is ignored. By default the index is always lost.</span>
<span class="sd"> options : dict</span>
<span class="sd"> All other options passed directly into Delta Lake.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> read_delta</span>
<span class="sd"> DataFrame.to_parquet</span>
<span class="sd"> DataFrame.to_table</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(dict(</span>
<span class="sd"> ... date=list(pd.date_range(&#39;2012-1-1 12:00:00&#39;, periods=3, freq=&#39;M&#39;)),</span>
<span class="sd"> ... country=[&#39;KR&#39;, &#39;US&#39;, &#39;JP&#39;],</span>
<span class="sd"> ... code=[1, 2 ,3]), columns=[&#39;date&#39;, &#39;country&#39;, &#39;code&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> date country code</span>
<span class="sd"> 0 2012-01-31 12:00:00 KR 1</span>
<span class="sd"> 1 2012-02-29 12:00:00 US 2</span>
<span class="sd"> 2 2012-03-31 12:00:00 JP 3</span>
<span class="sd"> Create a new Delta Lake table, partitioned by one column:</span>
<span class="sd"> &gt;&gt;&gt; df.to_delta(&#39;%s/to_delta/foo&#39; % path, partition_cols=&#39;date&#39;) # doctest: +SKIP</span>
<span class="sd"> Partitioned by two columns:</span>
<span class="sd"> &gt;&gt;&gt; df.to_delta(&#39;%s/to_delta/bar&#39; % path,</span>
<span class="sd"> ... partition_cols=[&#39;date&#39;, &#39;country&#39;]) # doctest: +SKIP</span>
<span class="sd"> Overwrite an existing table&#39;s partitions, using the &#39;replaceWhere&#39; capability in Delta:</span>
<span class="sd"> &gt;&gt;&gt; df.to_delta(&#39;%s/to_delta/bar&#39; % path,</span>
<span class="sd"> ... mode=&#39;overwrite&#39;, replaceWhere=&#39;date &gt;= &quot;2012-01-01&quot;&#39;) # doctest: +SKIP</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">index_col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="s2">&quot;If `index_col` is not specified for `to_delta`, &quot;</span>
<span class="s2">&quot;the existing index is lost when converting to Delta.&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="s2">&quot;options&quot;</span> <span class="ow">in</span> <span class="n">options</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">options</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;options&quot;</span><span class="p">),</span> <span class="nb">dict</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;options&quot;</span><span class="p">)</span> <span class="c1"># type: ignore[assignment]</span>
<span class="n">mode</span> <span class="o">=</span> <span class="n">validate_mode</span><span class="p">(</span><span class="n">mode</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">to_spark_io</span><span class="p">(</span>
<span class="n">path</span><span class="o">=</span><span class="n">path</span><span class="p">,</span>
<span class="n">mode</span><span class="o">=</span><span class="n">mode</span><span class="p">,</span>
<span class="nb">format</span><span class="o">=</span><span class="s2">&quot;delta&quot;</span><span class="p">,</span>
<span class="n">partition_cols</span><span class="o">=</span><span class="n">partition_cols</span><span class="p">,</span>
<span class="n">index_col</span><span class="o">=</span><span class="n">index_col</span><span class="p">,</span>
<span class="o">**</span><span class="n">options</span><span class="p">,</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_parquet"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_parquet.html#pyspark.pandas.DataFrame.to_parquet">[docs]</a> <span class="k">def</span> <span class="nf">to_parquet</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">path</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span>
<span class="n">mode</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;w&quot;</span><span class="p">,</span>
<span class="n">partition_cols</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">compression</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="n">index_col</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="o">**</span><span class="n">options</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</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;</span>
<span class="sd"> Write the DataFrame out as a Parquet file or directory.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> path : str, required</span>
<span class="sd"> Path to write to.</span>
<span class="sd"> mode : str</span>
<span class="sd"> Python write mode, default &#39;w&#39;.</span>
<span class="sd"> .. note:: mode can accept the strings for Spark writing mode.</span>
<span class="sd"> Such as &#39;append&#39;, &#39;overwrite&#39;, &#39;ignore&#39;, &#39;error&#39;, &#39;errorifexists&#39;.</span>
<span class="sd"> - &#39;append&#39; (equivalent to &#39;a&#39;): Append the new data to existing data.</span>
<span class="sd"> - &#39;overwrite&#39; (equivalent to &#39;w&#39;): Overwrite existing data.</span>
<span class="sd"> - &#39;ignore&#39;: Silently ignore this operation if data already exists.</span>
<span class="sd"> - &#39;error&#39; or &#39;errorifexists&#39;: Throw an exception if data already exists.</span>
<span class="sd"> partition_cols : str or list of str, optional, default None</span>
<span class="sd"> Names of partitioning columns</span>
<span class="sd"> compression : str {&#39;none&#39;, &#39;uncompressed&#39;, &#39;snappy&#39;, &#39;gzip&#39;, &#39;lzo&#39;, &#39;brotli&#39;, &#39;lz4&#39;, &#39;zstd&#39;}</span>
<span class="sd"> Compression codec to use when saving to file. If None is set, it uses the</span>
<span class="sd"> value specified in `spark.sql.parquet.compression.codec`.</span>
<span class="sd"> index_col: str or list of str, optional, default: None</span>
<span class="sd"> Column names to be used in Spark to represent pandas-on-Spark&#39;s index. The index name</span>
<span class="sd"> in pandas-on-Spark is ignored. By default the index is always lost.</span>
<span class="sd"> options : dict</span>
<span class="sd"> All other options passed directly into Spark&#39;s data source.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> read_parquet</span>
<span class="sd"> DataFrame.to_delta</span>
<span class="sd"> DataFrame.to_table</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(dict(</span>
<span class="sd"> ... date=list(pd.date_range(&#39;2012-1-1 12:00:00&#39;, periods=3, freq=&#39;M&#39;)),</span>
<span class="sd"> ... country=[&#39;KR&#39;, &#39;US&#39;, &#39;JP&#39;],</span>
<span class="sd"> ... code=[1, 2 ,3]), columns=[&#39;date&#39;, &#39;country&#39;, &#39;code&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> date country code</span>
<span class="sd"> 0 2012-01-31 12:00:00 KR 1</span>
<span class="sd"> 1 2012-02-29 12:00:00 US 2</span>
<span class="sd"> 2 2012-03-31 12:00:00 JP 3</span>
<span class="sd"> &gt;&gt;&gt; df.to_parquet(&#39;%s/to_parquet/foo.parquet&#39; % path, partition_cols=&#39;date&#39;)</span>
<span class="sd"> &gt;&gt;&gt; df.to_parquet(</span>
<span class="sd"> ... &#39;%s/to_parquet/foo.parquet&#39; % path,</span>
<span class="sd"> ... mode = &#39;overwrite&#39;,</span>
<span class="sd"> ... partition_cols=[&#39;date&#39;, &#39;country&#39;])</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> pandas API on Spark writes Parquet files into the directory, `path`, and writes</span>
<span class="sd"> multiple part files in the directory unlike pandas.</span>
<span class="sd"> pandas API on Spark respects HDFS&#39;s property such as &#39;fs.default.name&#39;.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">index_col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="s2">&quot;If `index_col` is not specified for `to_parquet`, &quot;</span>
<span class="s2">&quot;the existing index is lost when converting to Parquet.&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="s2">&quot;options&quot;</span> <span class="ow">in</span> <span class="n">options</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">options</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;options&quot;</span><span class="p">),</span> <span class="nb">dict</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;options&quot;</span><span class="p">)</span>
<span class="n">mode</span> <span class="o">=</span> <span class="n">validate_mode</span><span class="p">(</span><span class="n">mode</span><span class="p">)</span>
<span class="n">builder</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_spark</span><span class="p">(</span><span class="n">index_col</span><span class="o">=</span><span class="n">index_col</span><span class="p">)</span><span class="o">.</span><span class="n">write</span><span class="o">.</span><span class="n">mode</span><span class="p">(</span><span class="n">mode</span><span class="p">)</span>
<span class="k">if</span> <span class="n">partition_cols</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">builder</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="n">partition_cols</span><span class="p">)</span>
<span class="k">if</span> <span class="n">compression</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">builder</span><span class="o">.</span><span class="n">option</span><span class="p">(</span><span class="s2">&quot;compression&quot;</span><span class="p">,</span> <span class="n">compression</span><span class="p">)</span>
<span class="n">builder</span><span class="o">.</span><span class="n">options</span><span class="p">(</span><span class="o">**</span><span class="n">options</span><span class="p">)</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="s2">&quot;parquet&quot;</span><span class="p">)</span><span class="o">.</span><span class="n">save</span><span class="p">(</span><span class="n">path</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_orc"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_orc.html#pyspark.pandas.DataFrame.to_orc">[docs]</a> <span class="k">def</span> <span class="nf">to_orc</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">path</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span>
<span class="n">mode</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;w&quot;</span><span class="p">,</span>
<span class="n">partition_cols</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index_col</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="o">**</span><span class="n">options</span><span class="p">:</span> <span class="s2">&quot;OptionalPrimitiveType&quot;</span><span class="p">,</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;</span>
<span class="sd"> Write a DataFrame to the ORC format.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> path : str</span>
<span class="sd"> Path to write to.</span>
<span class="sd"> mode : str</span>
<span class="sd"> Python write mode, default &#39;w&#39;.</span>
<span class="sd"> .. note:: mode can accept the strings for Spark writing mode.</span>
<span class="sd"> Such as &#39;append&#39;, &#39;overwrite&#39;, &#39;ignore&#39;, &#39;error&#39;, &#39;errorifexists&#39;.</span>
<span class="sd"> - &#39;append&#39; (equivalent to &#39;a&#39;): Append the new data to existing data.</span>
<span class="sd"> - &#39;overwrite&#39; (equivalent to &#39;w&#39;): Overwrite existing data.</span>
<span class="sd"> - &#39;ignore&#39;: Silently ignore this operation if data already exists.</span>
<span class="sd"> - &#39;error&#39; or &#39;errorifexists&#39;: Throw an exception if data already exists.</span>
<span class="sd"> partition_cols : str or list of str, optional, default None</span>
<span class="sd"> Names of partitioning columns</span>
<span class="sd"> index_col: str or list of str, optional, default: None</span>
<span class="sd"> Column names to be used in Spark to represent pandas-on-Spark&#39;s index. The index name</span>
<span class="sd"> in pandas-on-Spark is ignored. By default the index is always lost.</span>
<span class="sd"> options : dict</span>
<span class="sd"> All other options passed directly into Spark&#39;s data source.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> read_orc</span>
<span class="sd"> DataFrame.to_delta</span>
<span class="sd"> DataFrame.to_parquet</span>
<span class="sd"> DataFrame.to_table</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(dict(</span>
<span class="sd"> ... date=list(pd.date_range(&#39;2012-1-1 12:00:00&#39;, periods=3, freq=&#39;M&#39;)),</span>
<span class="sd"> ... country=[&#39;KR&#39;, &#39;US&#39;, &#39;JP&#39;],</span>
<span class="sd"> ... code=[1, 2 ,3]), columns=[&#39;date&#39;, &#39;country&#39;, &#39;code&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> date country code</span>
<span class="sd"> 0 2012-01-31 12:00:00 KR 1</span>
<span class="sd"> 1 2012-02-29 12:00:00 US 2</span>
<span class="sd"> 2 2012-03-31 12:00:00 JP 3</span>
<span class="sd"> &gt;&gt;&gt; df.to_orc(&#39;%s/to_orc/foo.orc&#39; % path, partition_cols=&#39;date&#39;)</span>
<span class="sd"> &gt;&gt;&gt; df.to_orc(</span>
<span class="sd"> ... &#39;%s/to_orc/foo.orc&#39; % path,</span>
<span class="sd"> ... mode = &#39;overwrite&#39;,</span>
<span class="sd"> ... partition_cols=[&#39;date&#39;, &#39;country&#39;])</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> pandas API on Spark writes ORC files into the directory, `path`, and writes</span>
<span class="sd"> multiple part files in the directory unlike pandas.</span>
<span class="sd"> pandas API on Spark respects HDFS&#39;s property such as &#39;fs.default.name&#39;.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">index_col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="s2">&quot;If `index_col` is not specified for `to_orc`, &quot;</span>
<span class="s2">&quot;the existing index is lost when converting to ORC.&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="s2">&quot;options&quot;</span> <span class="ow">in</span> <span class="n">options</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">options</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;options&quot;</span><span class="p">),</span> <span class="nb">dict</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">options</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">options</span> <span class="o">=</span> <span class="n">options</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;options&quot;</span><span class="p">)</span> <span class="c1"># type: ignore[assignment]</span>
<span class="n">mode</span> <span class="o">=</span> <span class="n">validate_mode</span><span class="p">(</span><span class="n">mode</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">to_spark_io</span><span class="p">(</span>
<span class="n">path</span><span class="o">=</span><span class="n">path</span><span class="p">,</span>
<span class="n">mode</span><span class="o">=</span><span class="n">mode</span><span class="p">,</span>
<span class="nb">format</span><span class="o">=</span><span class="s2">&quot;orc&quot;</span><span class="p">,</span>
<span class="n">partition_cols</span><span class="o">=</span><span class="n">partition_cols</span><span class="p">,</span>
<span class="n">index_col</span><span class="o">=</span><span class="n">index_col</span><span class="p">,</span>
<span class="o">**</span><span class="n">options</span><span class="p">,</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_spark"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_spark.html#pyspark.pandas.DataFrame.to_spark">[docs]</a> <span class="k">def</span> <span class="nf">to_spark</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">index_col</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">PySparkDataFrame</span><span class="p">:</span>
<span class="k">if</span> <span class="n">index_col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="s2">&quot;If `index_col` is not specified for `to_spark`, &quot;</span>
<span class="s2">&quot;the existing index is lost when converting to Spark DataFrame.&quot;</span>
<span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_to_spark</span><span class="p">(</span><span class="n">index_col</span><span class="p">)</span></div>
<span class="n">to_spark</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">SparkFrameMethods</span><span class="o">.</span><span class="vm">__doc__</span>
<span class="k">def</span> <span class="nf">_to_spark</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">index_col</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">PySparkDataFrame</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Same as `to_spark()`, without issuing the advice log when `index_col` is not specified</span>
<span class="sd"> for internal usage.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">frame</span><span class="p">(</span><span class="n">index_col</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.to_pandas"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_pandas.html#pyspark.pandas.DataFrame.to_pandas">[docs]</a> <span class="k">def</span> <span class="nf">to_pandas</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a pandas DataFrame.</span>
<span class="sd"> .. note:: This method should only be used if the resulting pandas DataFrame is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(.2, .3), (.0, .6), (.6, .0), (.2, .1)],</span>
<span class="sd"> ... columns=[&#39;dogs&#39;, &#39;cats&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.to_pandas()</span>
<span class="sd"> dogs cats</span>
<span class="sd"> 0 0.2 0.3</span>
<span class="sd"> 1 0.0 0.6</span>
<span class="sd"> 2 0.6 0.0</span>
<span class="sd"> 3 0.2 0.1</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="s2">&quot;`to_pandas` loads all data into the driver&#39;s memory. &quot;</span>
<span class="s2">&quot;It should only be used if the resulting pandas DataFrame is expected to be small.&quot;</span>
<span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_to_pandas</span><span class="p">()</span></div>
<span class="k">def</span> <span class="nf">_to_pandas</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Same as `to_pandas()`, without issuing the advice log for internal usage.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">to_pandas_frame</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<div class="viewcode-block" id="DataFrame.assign"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.assign.html#pyspark.pandas.DataFrame.assign">[docs]</a> <span class="k">def</span> <span class="nf">assign</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Assign new columns to a DataFrame.</span>
<span class="sd"> Returns a new object with all original columns in addition to new ones.</span>
<span class="sd"> Existing columns that are re-assigned will be overwritten.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> **kwargs : dict of {str: callable, Series or Index}</span>
<span class="sd"> The column names are keywords. If the values are</span>
<span class="sd"> callable, they are computed on the DataFrame and</span>
<span class="sd"> assigned to the new columns. The callable must not</span>
<span class="sd"> change input DataFrame (though pandas-on-Spark doesn&#39;t check it).</span>
<span class="sd"> If the values are not callable, (e.g. a Series or a literal),</span>
<span class="sd"> they are simply assigned.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> A new DataFrame with the new columns in addition to</span>
<span class="sd"> all the existing columns.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;temp_c&#39;: [17.0, 25.0]},</span>
<span class="sd"> ... index=[&#39;Portland&#39;, &#39;Berkeley&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> temp_c</span>
<span class="sd"> Portland 17.0</span>
<span class="sd"> Berkeley 25.0</span>
<span class="sd"> Where the value is a callable, evaluated on `df`:</span>
<span class="sd"> &gt;&gt;&gt; df.assign(temp_f=lambda x: x.temp_c * 9 / 5 + 32)</span>
<span class="sd"> temp_c temp_f</span>
<span class="sd"> Portland 17.0 62.6</span>
<span class="sd"> Berkeley 25.0 77.0</span>
<span class="sd"> Alternatively, the same behavior can be achieved by directly</span>
<span class="sd"> referencing an existing Series or sequence and you can also</span>
<span class="sd"> create multiple columns within the same assign.</span>
<span class="sd"> &gt;&gt;&gt; assigned = df.assign(temp_f=df[&#39;temp_c&#39;] * 9 / 5 + 32,</span>
<span class="sd"> ... temp_k=df[&#39;temp_c&#39;] + 273.15,</span>
<span class="sd"> ... temp_idx=df.index)</span>
<span class="sd"> &gt;&gt;&gt; assigned[[&#39;temp_c&#39;, &#39;temp_f&#39;, &#39;temp_k&#39;, &#39;temp_idx&#39;]]</span>
<span class="sd"> temp_c temp_f temp_k temp_idx</span>
<span class="sd"> Portland 17.0 62.6 290.15 Portland</span>
<span class="sd"> Berkeley 25.0 77.0 298.15 Berkeley</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> Assigning multiple columns within the same ``assign`` is possible</span>
<span class="sd"> but you cannot refer to newly created or modified columns. This</span>
<span class="sd"> feature is supported in pandas for Python 3.6 and later but not in</span>
<span class="sd"> pandas-on-Spark. In pandas-on-Spark, all items are computed first,</span>
<span class="sd"> and then assigned.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_assign</span><span class="p">(</span><span class="n">kwargs</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_assign</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">kwargs</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.indexes</span> <span class="kn">import</span> <span class="n">MultiIndex</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">IndexOpsMixin</span>
<span class="n">Column</span> <span class="o">=</span> <span class="n">get_column_class</span><span class="p">()</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">is_invalid_assignee</span> <span class="o">=</span> <span class="p">(</span>
<span class="ow">not</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="p">(</span><span class="n">IndexOpsMixin</span><span class="p">,</span> <span class="n">Column</span><span class="p">))</span> <span class="ow">or</span> <span class="nb">callable</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="ow">or</span> <span class="n">is_scalar</span><span class="p">(</span><span class="n">v</span><span class="p">))</span>
<span class="p">)</span> <span class="ow">or</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">MultiIndex</span><span class="p">)</span>
<span class="k">if</span> <span class="n">is_invalid_assignee</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Column assignment doesn&#39;t support type &quot;</span> <span class="s2">&quot;</span><span class="si">{0}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">v</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">callable</span><span class="p">(</span><span class="n">v</span><span class="p">):</span>
<span class="n">kwargs</span><span class="p">[</span><span class="n">k</span><span class="p">]</span> <span class="o">=</span> <span class="n">v</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span>
<span class="n">pairs</span> <span class="o">=</span> <span class="p">{</span>
<span class="p">(</span><span class="n">k</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">k</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">k</span><span class="p">,)):</span> <span class="p">(</span>
<span class="p">(</span><span class="n">v</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="p">,</span> <span class="n">v</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">IndexOpsMixin</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">MultiIndex</span><span class="p">)</span>
<span class="k">else</span> <span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="n">Column</span><span class="p">)</span>
<span class="k">else</span> <span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">v</span><span class="p">),</span> <span class="kc">None</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span>
<span class="n">scols</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)):</span>
<span class="k">if</span> <span class="n">label</span><span class="p">[:</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">-</span> <span class="n">i</span><span class="p">]</span> <span class="ow">in</span> <span class="n">pairs</span><span class="p">:</span>
<span class="n">scol</span><span class="p">,</span> <span class="n">field</span> <span class="o">=</span> <span class="n">pairs</span><span class="p">[</span><span class="n">label</span><span class="p">[:</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">-</span> <span class="n">i</span><span class="p">]]</span>
<span class="n">name</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name</span><span class="p">)</span>
<span class="k">if</span> <span class="n">field</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">field</span> <span class="o">=</span> <span class="n">field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">name</span><span class="p">)</span>
<span class="k">break</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">field</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">field_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">scols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol</span><span class="p">)</span>
<span class="n">data_fields</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">field</span><span class="p">)</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="p">(</span><span class="n">scol</span><span class="p">,</span> <span class="n">field</span><span class="p">)</span> <span class="ow">in</span> <span class="n">pairs</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="nb">set</span><span class="p">(</span><span class="n">i</span><span class="p">[:</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)]</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">):</span>
<span class="n">name</span> <span class="o">=</span> <span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">scols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name</span><span class="p">))</span>
<span class="k">if</span> <span class="n">field</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">field</span> <span class="o">=</span> <span class="n">field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">name</span><span class="p">)</span>
<span class="n">data_fields</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">field</span><span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">level</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="nb">tuple</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">+</span> <span class="p">([</span><span class="s2">&quot;&quot;</span><span class="p">]</span> <span class="o">*</span> <span class="p">(</span><span class="n">level</span> <span class="o">-</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">))))</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span>
<span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span>
<span class="n">cast</span><span class="p">(</span><span class="n">Sequence</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">,</span> <span class="s2">&quot;Series&quot;</span><span class="p">]],</span> <span class="n">scols</span><span class="p">),</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_fields</span><span class="o">=</span><span class="n">data_fields</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">internal</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.from_records"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.from_records.html#pyspark.pandas.DataFrame.from_records">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_records</span><span class="p">(</span>
<span class="n">data</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">tuple</span><span class="p">],</span> <span class="nb">dict</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">],</span>
<span class="n">index</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">list</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">exclude</span><span class="p">:</span> <span class="nb">list</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">columns</span><span class="p">:</span> <span class="nb">list</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">coerce_float</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">nrows</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Convert structured or recorded ndarray to DataFrame.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> data : ndarray (structured dtype), list of tuples, dict, or DataFrame</span>
<span class="sd"> .. deprecated:: 4.0.0</span>
<span class="sd"> Passing a DataFrame is deprecated.</span>
<span class="sd"> index : string, list of fields, array-like</span>
<span class="sd"> Field of array to use as the index, alternately a specific set of input labels to use</span>
<span class="sd"> exclude : sequence, default None</span>
<span class="sd"> Columns or fields to exclude</span>
<span class="sd"> columns : sequence, default None</span>
<span class="sd"> Column names to use. If the passed data do not have names associated with them, this</span>
<span class="sd"> argument provides names for the columns. Otherwise this argument indicates the order of</span>
<span class="sd"> the columns in the result (any names not found in the data will become all-NA columns)</span>
<span class="sd"> coerce_float : boolean, default False</span>
<span class="sd"> Attempt to convert values of non-string, non-numeric objects (like decimal.Decimal) to</span>
<span class="sd"> floating point, useful for SQL result sets</span>
<span class="sd"> nrows : int, default None</span>
<span class="sd"> Number of rows to read if data is an iterator</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> df : DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> Use dict as input</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame.from_records({&#39;A&#39;: [1, 2, 3]})</span>
<span class="sd"> A</span>
<span class="sd"> 0 1</span>
<span class="sd"> 1 2</span>
<span class="sd"> 2 3</span>
<span class="sd"> Use list of tuples as input</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame.from_records([(1, 2), (3, 4)])</span>
<span class="sd"> 0 1</span>
<span class="sd"> 0 1 2</span>
<span class="sd"> 1 3 4</span>
<span class="sd"> Use NumPy array as input</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame.from_records(np.eye(3))</span>
<span class="sd"> 0 1 2</span>
<span class="sd"> 0 1.0 0.0 0.0</span>
<span class="sd"> 1 0.0 1.0 0.0</span>
<span class="sd"> 2 0.0 0.0 1.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">from_records</span><span class="p">(</span><span class="n">data</span><span class="p">,</span> <span class="n">index</span><span class="p">,</span> <span class="n">exclude</span><span class="p">,</span> <span class="n">columns</span><span class="p">,</span> <span class="n">coerce_float</span><span class="p">,</span> <span class="n">nrows</span><span class="p">)</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.to_records"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.to_records.html#pyspark.pandas.DataFrame.to_records">[docs]</a> <span class="k">def</span> <span class="nf">to_records</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">column_dtypes</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">]]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index_dtypes</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">]]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">np</span><span class="o">.</span><span class="n">recarray</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Convert DataFrame to a NumPy record array.</span>
<span class="sd"> Index will be included as the first field of the record array if</span>
<span class="sd"> requested.</span>
<span class="sd"> .. note:: This method should only be used if the resulting NumPy ndarray is</span>
<span class="sd"> expected to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> index : bool, default True</span>
<span class="sd"> Include index in resulting record array, stored in &#39;index&#39;</span>
<span class="sd"> field or using the index label, if set.</span>
<span class="sd"> column_dtypes : str, type, dict, default None</span>
<span class="sd"> If a string or type, the data type to store all columns. If</span>
<span class="sd"> a dictionary, a mapping of column names and indices (zero-indexed)</span>
<span class="sd"> to specific data types.</span>
<span class="sd"> index_dtypes : str, type, dict, default None</span>
<span class="sd"> If a string or type, the data type to store all index levels. If</span>
<span class="sd"> a dictionary, a mapping of index level names and indices</span>
<span class="sd"> (zero-indexed) to specific data types.</span>
<span class="sd"> This mapping is applied only if `index=True`.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> numpy.recarray</span>
<span class="sd"> NumPy ndarray with the DataFrame labels as fields and each row</span>
<span class="sd"> of the DataFrame as entries.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.from_records: Convert structured or record ndarray</span>
<span class="sd"> to DataFrame.</span>
<span class="sd"> numpy.recarray: An ndarray that allows field access using</span>
<span class="sd"> attributes, analogous to typed columns in a</span>
<span class="sd"> spreadsheet.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [1, 2], &#39;B&#39;: [0.5, 0.75]},</span>
<span class="sd"> ... index=[&#39;a&#39;, &#39;b&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> a 1 0.50</span>
<span class="sd"> b 2 0.75</span>
<span class="sd"> &gt;&gt;&gt; df.to_records() # doctest: +SKIP</span>
<span class="sd"> rec.array([(&#39;a&#39;, 1, 0.5 ), (&#39;b&#39;, 2, 0.75)],</span>
<span class="sd"> dtype=[(&#39;index&#39;, &#39;O&#39;), (&#39;A&#39;, &#39;&lt;i8&#39;), (&#39;B&#39;, &#39;&lt;f8&#39;)])</span>
<span class="sd"> The index can be excluded from the record array:</span>
<span class="sd"> &gt;&gt;&gt; df.to_records(index=False) # doctest: +SKIP</span>
<span class="sd"> rec.array([(1, 0.5 ), (2, 0.75)],</span>
<span class="sd"> dtype=[(&#39;A&#39;, &#39;&lt;i8&#39;), (&#39;B&#39;, &#39;&lt;f8&#39;)])</span>
<span class="sd"> Specification of dtype for columns is new in pandas 0.24.0.</span>
<span class="sd"> Data types can be specified for the columns:</span>
<span class="sd"> &gt;&gt;&gt; df.to_records(column_dtypes={&quot;A&quot;: &quot;int32&quot;}) # doctest: +SKIP</span>
<span class="sd"> rec.array([(&#39;a&#39;, 1, 0.5 ), (&#39;b&#39;, 2, 0.75)],</span>
<span class="sd"> dtype=[(&#39;index&#39;, &#39;O&#39;), (&#39;A&#39;, &#39;&lt;i4&#39;), (&#39;B&#39;, &#39;&lt;f8&#39;)])</span>
<span class="sd"> Specification of dtype for index is new in pandas 0.24.0.</span>
<span class="sd"> Data types can also be specified for the index:</span>
<span class="sd"> &gt;&gt;&gt; df.to_records(index_dtypes=&quot;&lt;S2&quot;) # doctest: +SKIP</span>
<span class="sd"> rec.array([(b&#39;a&#39;, 1, 0.5 ), (b&#39;b&#39;, 2, 0.75)],</span>
<span class="sd"> dtype=[(&#39;index&#39;, &#39;S2&#39;), (&#39;A&#39;, &#39;&lt;i8&#39;), (&#39;B&#39;, &#39;&lt;f8&#39;)])</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">args</span> <span class="o">=</span> <span class="nb">locals</span><span class="p">()</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span>
<span class="k">return</span> <span class="n">validate_arguments_and_invoke_function</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">(),</span> <span class="bp">self</span><span class="o">.</span><span class="n">to_records</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">to_records</span><span class="p">,</span> <span class="n">args</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.copy"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.copy.html#pyspark.pandas.DataFrame.copy">[docs]</a> <span class="k">def</span> <span class="nf">copy</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">deep</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Make a copy of this object&#39;s indices and data.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> deep : bool, default True</span>
<span class="sd"> this parameter is not supported but just dummy parameter to match pandas.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> copy : DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;x&#39;: [1, 2], &#39;y&#39;: [3, 4], &#39;z&#39;: [5, 6], &#39;w&#39;: [7, 8]},</span>
<span class="sd"> ... columns=[&#39;x&#39;, &#39;y&#39;, &#39;z&#39;, &#39;w&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> x y z w</span>
<span class="sd"> 0 1 3 5 7</span>
<span class="sd"> 1 2 4 6 8</span>
<span class="sd"> &gt;&gt;&gt; df_copy = df.copy()</span>
<span class="sd"> &gt;&gt;&gt; df_copy</span>
<span class="sd"> x y z w</span>
<span class="sd"> 0 1 3 5 7</span>
<span class="sd"> 1 2 4 6 8</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.dropna"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.dropna.html#pyspark.pandas.DataFrame.dropna">[docs]</a> <span class="k">def</span> <span class="nf">dropna</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="n">how</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;any&quot;</span><span class="p">,</span>
<span class="n">thresh</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">subset</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Remove missing values.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> axis : {0 or &#39;index&#39;}, default 0</span>
<span class="sd"> Determine if rows or columns which contain missing values are</span>
<span class="sd"> removed.</span>
<span class="sd"> * 0, or &#39;index&#39; : Drop rows which contain missing values.</span>
<span class="sd"> how : {&#39;any&#39;, &#39;all&#39;}, default &#39;any&#39;</span>
<span class="sd"> Determine if row or column is removed from DataFrame, when we have</span>
<span class="sd"> at least one NA or all NA.</span>
<span class="sd"> * &#39;any&#39; : If any NA values are present, drop that row or column.</span>
<span class="sd"> * &#39;all&#39; : If all values are NA, drop that row or column.</span>
<span class="sd"> thresh : int, optional</span>
<span class="sd"> Require that many non-NA values.</span>
<span class="sd"> subset : array-like, optional</span>
<span class="sd"> Labels along other axis to consider, e.g. if you are dropping rows</span>
<span class="sd"> these would be a list of columns to include.</span>
<span class="sd"> inplace : bool, default False</span>
<span class="sd"> If True, do operation inplace and return None.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame with NA entries dropped from it.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.drop : Drop specified labels from columns.</span>
<span class="sd"> DataFrame.isnull: Indicate missing values.</span>
<span class="sd"> DataFrame.notnull : Indicate existing (non-missing) values.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&quot;name&quot;: [&#39;Alfred&#39;, &#39;Batman&#39;, &#39;Catwoman&#39;],</span>
<span class="sd"> ... &quot;toy&quot;: [None, &#39;Batmobile&#39;, &#39;Bullwhip&#39;],</span>
<span class="sd"> ... &quot;born&quot;: [None, &quot;1940-04-25&quot;, None]},</span>
<span class="sd"> ... columns=[&#39;name&#39;, &#39;toy&#39;, &#39;born&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> name toy born</span>
<span class="sd"> 0 Alfred None None</span>
<span class="sd"> 1 Batman Batmobile 1940-04-25</span>
<span class="sd"> 2 Catwoman Bullwhip None</span>
<span class="sd"> Drop the rows where at least one element is missing.</span>
<span class="sd"> &gt;&gt;&gt; df.dropna()</span>
<span class="sd"> name toy born</span>
<span class="sd"> 1 Batman Batmobile 1940-04-25</span>
<span class="sd"> Drop the columns where at least one element is missing.</span>
<span class="sd"> &gt;&gt;&gt; df.dropna(axis=&#39;columns&#39;)</span>
<span class="sd"> name</span>
<span class="sd"> 0 Alfred</span>
<span class="sd"> 1 Batman</span>
<span class="sd"> 2 Catwoman</span>
<span class="sd"> Drop the rows where all elements are missing.</span>
<span class="sd"> &gt;&gt;&gt; df.dropna(how=&#39;all&#39;)</span>
<span class="sd"> name toy born</span>
<span class="sd"> 0 Alfred None None</span>
<span class="sd"> 1 Batman Batmobile 1940-04-25</span>
<span class="sd"> 2 Catwoman Bullwhip None</span>
<span class="sd"> Keep only the rows with at least 2 non-NA values.</span>
<span class="sd"> &gt;&gt;&gt; df.dropna(thresh=2)</span>
<span class="sd"> name toy born</span>
<span class="sd"> 1 Batman Batmobile 1940-04-25</span>
<span class="sd"> 2 Catwoman Bullwhip None</span>
<span class="sd"> Define in which columns to look for missing values.</span>
<span class="sd"> &gt;&gt;&gt; df.dropna(subset=[&#39;name&#39;, &#39;born&#39;])</span>
<span class="sd"> name toy born</span>
<span class="sd"> 1 Batman Batmobile 1940-04-25</span>
<span class="sd"> Keep the DataFrame with valid entries in the same variable.</span>
<span class="sd"> &gt;&gt;&gt; df.dropna(inplace=True)</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> name toy born</span>
<span class="sd"> 1 Batman Batmobile 1940-04-25</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">thresh</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">how</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;must specify how or thresh&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">how</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">(</span><span class="s2">&quot;any&quot;</span><span class="p">,</span> <span class="s2">&quot;all&quot;</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;invalid how option: </span><span class="si">{h}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">h</span><span class="o">=</span><span class="n">how</span><span class="p">))</span>
<span class="n">labels</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]]</span>
<span class="k">if</span> <span class="n">subset</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">subset</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span>
<span class="n">labels</span> <span class="o">=</span> <span class="p">[(</span><span class="n">subset</span><span class="p">,)]</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">subset</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span>
<span class="n">labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">subset</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">sub</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">sub</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">sub</span><span class="p">,)</span> <span class="k">for</span> <span class="n">sub</span> <span class="ow">in</span> <span class="n">subset</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">labels</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">if</span> <span class="n">labels</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">invalids</span> <span class="o">=</span> <span class="p">[</span><span class="n">label</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">labels</span> <span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">invalids</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="n">invalids</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">cnt</span> <span class="o">=</span> <span class="n">reduce</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">:</span> <span class="n">x</span> <span class="o">+</span> <span class="n">y</span><span class="p">,</span>
<span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">notna</span><span class="p">()</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">labels</span>
<span class="p">],</span>
<span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="mi">0</span><span class="p">),</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">thresh</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">pred</span> <span class="o">=</span> <span class="n">cnt</span> <span class="o">&gt;=</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="nb">int</span><span class="p">(</span><span class="n">thresh</span><span class="p">))</span>
<span class="k">elif</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;any&quot;</span><span class="p">:</span>
<span class="n">pred</span> <span class="o">=</span> <span class="n">cnt</span> <span class="o">==</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">labels</span><span class="p">))</span>
<span class="k">elif</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;all&quot;</span><span class="p">:</span>
<span class="n">pred</span> <span class="o">=</span> <span class="n">cnt</span> <span class="o">&gt;</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="mi">0</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">pred</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">assert</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">1</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="k">if</span> <span class="n">labels</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">lbl</span><span class="p">)</span> <span class="o">!=</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_level</span> <span class="k">for</span> <span class="n">lbl</span> <span class="ow">in</span> <span class="n">labels</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;The length of each subset must be the same as the index size.&quot;</span>
<span class="p">)</span>
<span class="n">cond</span> <span class="o">=</span> <span class="n">reduce</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">:</span> <span class="n">x</span> <span class="o">|</span> <span class="n">y</span><span class="p">,</span>
<span class="p">[</span>
<span class="n">reduce</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">:</span> <span class="n">x</span> <span class="o">&amp;</span> <span class="n">y</span><span class="p">,</span>
<span class="p">[</span>
<span class="n">scol</span> <span class="o">==</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">part</span><span class="p">)</span>
<span class="k">for</span> <span class="n">part</span><span class="p">,</span> <span class="n">scol</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">lbl</span><span class="p">,</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">)</span>
<span class="p">],</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">lbl</span> <span class="ow">in</span> <span class="n">labels</span>
<span class="p">],</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">cond</span><span class="p">)</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="n">null_counts</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">cond</span> <span class="o">=</span> <span class="n">psser</span><span class="o">.</span><span class="n">isnull</span><span class="p">()</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span>
<span class="n">null_counts</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">sum</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="o">~</span><span class="n">cond</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="mi">0</span><span class="p">))</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="p">)</span>
<span class="n">counts</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">null_counts</span> <span class="o">+</span> <span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">count</span><span class="p">(</span><span class="s2">&quot;*&quot;</span><span class="p">)])</span><span class="o">.</span><span class="n">head</span><span class="p">()</span>
<span class="k">if</span> <span class="n">thresh</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">label</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">cnt</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">counts</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="n">cnt</span> <span class="ow">or</span> <span class="mi">0</span><span class="p">)</span> <span class="o">&gt;=</span> <span class="nb">int</span><span class="p">(</span><span class="n">thresh</span><span class="p">)</span>
<span class="p">]</span>
<span class="k">elif</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;any&quot;</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">label</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">cnt</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">counts</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="n">cnt</span> <span class="ow">or</span> <span class="mi">0</span><span class="p">)</span> <span class="o">==</span> <span class="n">counts</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span>
<span class="p">]</span>
<span class="k">elif</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;all&quot;</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">label</span> <span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">cnt</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">counts</span><span class="p">)</span> <span class="k">if</span> <span class="p">(</span><span class="n">cnt</span> <span class="ow">or</span> <span class="mi">0</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span>
<span class="p">]</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="n">column_labels</span><span class="p">]</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span></div>
<span class="c1"># TODO: add &#39;limit&#39; when value parameter exists</span>
<div class="viewcode-block" id="DataFrame.fillna"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.fillna.html#pyspark.pandas.DataFrame.fillna">[docs]</a> <span class="k">def</span> <span class="nf">fillna</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">value</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Any</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">method</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="n">axis</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Axis</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">limit</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Fill NA/NaN values.</span>
<span class="sd"> .. note:: the current implementation of &#39;method&#39; parameter in fillna uses Spark&#39;s Window</span>
<span class="sd"> without specifying partition specification. This leads to moving all data into</span>
<span class="sd"> a single partition in a single machine and could cause serious</span>
<span class="sd"> performance degradation. Avoid this method with very large datasets.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> value : scalar, dict, Series</span>
<span class="sd"> Value to use to fill holes. alternately a dict/Series of values</span>
<span class="sd"> specifying which value to use for each column.</span>
<span class="sd"> DataFrame is not supported.</span>
<span class="sd"> method : {&#39;backfill&#39;, &#39;bfill&#39;, &#39;pad&#39;, &#39;ffill&#39;, None}, default None</span>
<span class="sd"> Method to use for filling holes in reindexed Series pad / ffill: propagate last valid</span>
<span class="sd"> observation forward to next valid backfill / bfill:</span>
<span class="sd"> use NEXT valid observation to fill gap</span>
<span class="sd"> .. deprecated:: 4.0.0</span>
<span class="sd"> axis : {0 or `index`}</span>
<span class="sd"> 1 and `columns` are not supported.</span>
<span class="sd"> inplace : boolean, default False</span>
<span class="sd"> Fill in place (do not create a new object)</span>
<span class="sd"> limit : int, default None</span>
<span class="sd"> If method is specified, this is the maximum number of consecutive NaN values to</span>
<span class="sd"> forward/backward fill. In other words, if there is a gap with more than this number of</span>
<span class="sd"> consecutive NaNs, it will only be partially filled. If method is not specified,</span>
<span class="sd"> this is the maximum number of entries along the entire axis where NaNs will be filled.</span>
<span class="sd"> Must be greater than 0 if not None</span>
<span class="sd"> .. deprecated:: 4.0.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame with NA entries filled.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({</span>
<span class="sd"> ... &#39;A&#39;: [None, 3, None, None],</span>
<span class="sd"> ... &#39;B&#39;: [2, 4, None, 3],</span>
<span class="sd"> ... &#39;C&#39;: [None, None, None, 1],</span>
<span class="sd"> ... &#39;D&#39;: [0, 1, 5, 4]</span>
<span class="sd"> ... },</span>
<span class="sd"> ... columns=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;, &#39;D&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C D</span>
<span class="sd"> 0 NaN 2.0 NaN 0</span>
<span class="sd"> 1 3.0 4.0 NaN 1</span>
<span class="sd"> 2 NaN NaN NaN 5</span>
<span class="sd"> 3 NaN 3.0 1.0 4</span>
<span class="sd"> Replace all NaN elements with 0s.</span>
<span class="sd"> &gt;&gt;&gt; df.fillna(0)</span>
<span class="sd"> A B C D</span>
<span class="sd"> 0 0.0 2.0 0.0 0</span>
<span class="sd"> 1 3.0 4.0 0.0 1</span>
<span class="sd"> 2 0.0 0.0 0.0 5</span>
<span class="sd"> 3 0.0 3.0 1.0 4</span>
<span class="sd"> We can also propagate non-null values forward or backward.</span>
<span class="sd"> &gt;&gt;&gt; df.fillna(method=&#39;ffill&#39;)</span>
<span class="sd"> A B C D</span>
<span class="sd"> 0 NaN 2.0 NaN 0</span>
<span class="sd"> 1 3.0 4.0 NaN 1</span>
<span class="sd"> 2 3.0 4.0 NaN 5</span>
<span class="sd"> 3 3.0 3.0 1.0 4</span>
<span class="sd"> Replace all NaN elements in column &#39;A&#39;, &#39;B&#39;, &#39;C&#39;, and &#39;D&#39;, with 0, 1,</span>
<span class="sd"> 2, and 3 respectively.</span>
<span class="sd"> &gt;&gt;&gt; values = {&#39;A&#39;: 0, &#39;B&#39;: 1, &#39;C&#39;: 2, &#39;D&#39;: 3}</span>
<span class="sd"> &gt;&gt;&gt; df.fillna(value=values)</span>
<span class="sd"> A B C D</span>
<span class="sd"> 0 0.0 2.0 2.0 0</span>
<span class="sd"> 1 3.0 4.0 2.0 1</span>
<span class="sd"> 2 0.0 1.0 2.0 5</span>
<span class="sd"> 3 0.0 3.0 1.0 4</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;fillna currently only works for axis=0 or axis=&#39;index&#39;&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">value</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">value</span><span class="p">,</span> <span class="p">(</span><span class="nb">float</span><span class="p">,</span> <span class="nb">int</span><span class="p">,</span> <span class="nb">str</span><span class="p">,</span> <span class="nb">bool</span><span class="p">,</span> <span class="nb">dict</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Unsupported type </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">value</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">)</span>
<span class="k">if</span> <span class="n">limit</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;limit parameter for value is not support now&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="n">value</span> <span class="o">=</span> <span class="n">value</span><span class="o">.</span><span class="n">to_dict</span><span class="p">()</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">value</span><span class="o">.</span><span class="n">values</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">v</span><span class="p">,</span> <span class="p">(</span><span class="nb">float</span><span class="p">,</span> <span class="nb">int</span><span class="p">,</span> <span class="nb">str</span><span class="p">,</span> <span class="nb">bool</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Unsupported type </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">v</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">)</span>
<span class="n">value</span> <span class="o">=</span> <span class="p">{</span><span class="n">k</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">k</span><span class="p">)</span> <span class="k">else</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">value</span><span class="o">.</span><span class="n">items</span><span class="p">()}</span>
<span class="k">def</span> <span class="nf">op</span><span class="p">(</span><span class="n">psser</span><span class="p">:</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">:</span>
<span class="n">label</span> <span class="o">=</span> <span class="n">psser</span><span class="o">.</span><span class="n">_column_label</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">value</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">if</span> <span class="n">k</span> <span class="o">==</span> <span class="n">label</span><span class="p">[:</span> <span class="nb">len</span><span class="p">(</span><span class="n">k</span><span class="p">)]:</span>
<span class="k">return</span> <span class="n">psser</span><span class="o">.</span><span class="n">_fillna</span><span class="p">(</span>
<span class="n">value</span><span class="o">=</span><span class="n">value</span><span class="p">[</span><span class="n">k</span><span class="p">],</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">limit</span><span class="o">=</span><span class="n">limit</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psser</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">def</span> <span class="nf">op</span><span class="p">(</span><span class="n">psser</span><span class="p">:</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psser</span><span class="o">.</span><span class="n">_fillna</span><span class="p">(</span><span class="n">value</span><span class="o">=</span><span class="n">value</span><span class="p">,</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">limit</span><span class="o">=</span><span class="n">limit</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">method</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;DataFrame.fillna with &#39;method&#39; is deprecated and will raise in a future version. &quot;</span>
<span class="s2">&quot;Use DataFrame.ffill() or DataFrame.bfill() instead.&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">def</span> <span class="nf">op</span><span class="p">(</span><span class="n">psser</span><span class="p">:</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psser</span><span class="o">.</span><span class="n">_fillna</span><span class="p">(</span><span class="n">value</span><span class="o">=</span><span class="n">value</span><span class="p">,</span> <span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">limit</span><span class="o">=</span><span class="n">limit</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Must specify a fillna &#39;value&#39; or &#39;method&#39; parameter.&quot;</span><span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="n">op</span><span class="p">,</span> <span class="n">should_resolve</span><span class="o">=</span><span class="p">(</span><span class="n">method</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">))</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">,</span> <span class="n">check_same_anchor</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.interpolate"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.interpolate.html#pyspark.pandas.DataFrame.interpolate">[docs]</a> <span class="k">def</span> <span class="nf">interpolate</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">method</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;linear&quot;</span><span class="p">,</span>
<span class="n">limit</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">limit_direction</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="n">limit_area</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="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">if</span> <span class="n">method</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">[</span><span class="s2">&quot;linear&quot;</span><span class="p">]:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;interpolate currently works only for method=&#39;linear&#39;&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="n">limit</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">)</span> <span class="ow">and</span> <span class="p">(</span><span class="ow">not</span> <span class="n">limit</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;limit must be &gt; 0.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="n">limit_direction</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">)</span> <span class="ow">and</span> <span class="p">(</span>
<span class="n">limit_direction</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">[</span><span class="s2">&quot;forward&quot;</span><span class="p">,</span> <span class="s2">&quot;backward&quot;</span><span class="p">,</span> <span class="s2">&quot;both&quot;</span><span class="p">]</span>
<span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;invalid limit_direction: &#39;</span><span class="si">{}</span><span class="s2">&#39;&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">limit_direction</span><span class="p">))</span>
<span class="k">if</span> <span class="p">(</span><span class="n">limit_area</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">)</span> <span class="ow">and</span> <span class="p">(</span><span class="n">limit_area</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">[</span><span class="s2">&quot;inside&quot;</span><span class="p">,</span> <span class="s2">&quot;outside&quot;</span><span class="p">]):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;invalid limit_area: &#39;</span><span class="si">{}</span><span class="s2">&#39;&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">limit_area</span><span class="p">))</span>
<span class="k">for</span> <span class="n">dtype</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">dtypes</span><span class="o">.</span><span class="n">values</span><span class="p">:</span>
<span class="k">if</span> <span class="n">dtype</span> <span class="o">==</span> <span class="s2">&quot;object&quot;</span><span class="p">:</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;DataFrame.interpolate with object dtype is deprecated and will raise in a &quot;</span>
<span class="s2">&quot;future version. Convert to a specific numeric type before interpolating.&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">numeric_col_names</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">)):</span>
<span class="n">numeric_col_names</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">name</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">numeric_col_names</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Cannot interpolate with all object-dtype columns in the DataFrame. &quot;</span>
<span class="s2">&quot;Try setting at least one column to a numeric dtype.&quot;</span>
<span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="n">numeric_col_names</span><span class="p">]</span>
<span class="k">return</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">_interpolate</span><span class="p">(</span>
<span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="n">limit</span><span class="o">=</span><span class="n">limit</span><span class="p">,</span> <span class="n">limit_direction</span><span class="o">=</span><span class="n">limit_direction</span><span class="p">,</span> <span class="n">limit_area</span><span class="o">=</span><span class="n">limit_area</span>
<span class="p">),</span>
<span class="n">should_resolve</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.replace"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.replace.html#pyspark.pandas.DataFrame.replace">[docs]</a> <span class="k">def</span> <span class="nf">replace</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">to_replace</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">List</span><span class="p">,</span> <span class="n">Tuple</span><span class="p">,</span> <span class="n">Dict</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">value</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Any</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">limit</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">regex</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">method</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;pad&quot;</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns a new DataFrame replacing a value with another value.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> to_replace : int, float, string, list, tuple or dict</span>
<span class="sd"> Value to be replaced.</span>
<span class="sd"> value : int, float, string, list or tuple</span>
<span class="sd"> Value to use to replace holes. The replacement value must be an int, float,</span>
<span class="sd"> or string.</span>
<span class="sd"> If value is a list or tuple, value should be of the same length with to_replace.</span>
<span class="sd"> inplace : boolean, default False</span>
<span class="sd"> Fill in place (do not create a new object)</span>
<span class="sd"> limit : int, default None</span>
<span class="sd"> Maximum size gap to forward or backward fill.</span>
<span class="sd"> .. deprecated:: 4.0.0</span>
<span class="sd"> regex : bool or str, default False</span>
<span class="sd"> Whether to interpret to_replace and/or value as regular expressions.</span>
<span class="sd"> If this is True then to_replace must be a string.</span>
<span class="sd"> Alternatively, this could be a regular expression in which case to_replace must be None.</span>
<span class="sd"> method : &#39;pad&#39;, default None</span>
<span class="sd"> The method to use when for replacement, when to_replace is a scalar,</span>
<span class="sd"> list or tuple and value is None.</span>
<span class="sd"> .. deprecated:: 4.0.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Object after replacement.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&quot;name&quot;: [&#39;Ironman&#39;, &#39;Captain America&#39;, &#39;Thor&#39;, &#39;Hulk&#39;],</span>
<span class="sd"> ... &quot;weapon&quot;: [&#39;Mark-45&#39;, &#39;Shield&#39;, &#39;Mjolnir&#39;, &#39;Smash&#39;]},</span>
<span class="sd"> ... columns=[&#39;name&#39;, &#39;weapon&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> name weapon</span>
<span class="sd"> 0 Ironman Mark-45</span>
<span class="sd"> 1 Captain America Shield</span>
<span class="sd"> 2 Thor Mjolnir</span>
<span class="sd"> 3 Hulk Smash</span>
<span class="sd"> Scalar `to_replace` and `value`</span>
<span class="sd"> &gt;&gt;&gt; df.replace(&#39;Ironman&#39;, &#39;War-Machine&#39;)</span>
<span class="sd"> name weapon</span>
<span class="sd"> 0 War-Machine Mark-45</span>
<span class="sd"> 1 Captain America Shield</span>
<span class="sd"> 2 Thor Mjolnir</span>
<span class="sd"> 3 Hulk Smash</span>
<span class="sd"> List like `to_replace` and `value`</span>
<span class="sd"> &gt;&gt;&gt; df.replace([&#39;Ironman&#39;, &#39;Captain America&#39;], [&#39;Rescue&#39;, &#39;Hawkeye&#39;], inplace=True)</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> name weapon</span>
<span class="sd"> 0 Rescue Mark-45</span>
<span class="sd"> 1 Hawkeye Shield</span>
<span class="sd"> 2 Thor Mjolnir</span>
<span class="sd"> 3 Hulk Smash</span>
<span class="sd"> Dicts can be used to specify different replacement values for different existing values</span>
<span class="sd"> To use a dict in this way the value parameter should be None</span>
<span class="sd"> &gt;&gt;&gt; df.replace({&#39;Mjolnir&#39;: &#39;Stormbuster&#39;})</span>
<span class="sd"> name weapon</span>
<span class="sd"> 0 Rescue Mark-45</span>
<span class="sd"> 1 Hawkeye Shield</span>
<span class="sd"> 2 Thor Stormbuster</span>
<span class="sd"> 3 Hulk Smash</span>
<span class="sd"> Dict can specify that different values should be replaced in different columns</span>
<span class="sd"> The value parameter should not be None in this case</span>
<span class="sd"> &gt;&gt;&gt; df.replace({&#39;weapon&#39;: &#39;Mjolnir&#39;}, &#39;Stormbuster&#39;)</span>
<span class="sd"> name weapon</span>
<span class="sd"> 0 Rescue Mark-45</span>
<span class="sd"> 1 Hawkeye Shield</span>
<span class="sd"> 2 Thor Stormbuster</span>
<span class="sd"> 3 Hulk Smash</span>
<span class="sd"> Nested dictionaries</span>
<span class="sd"> The value parameter should be None to use a nested dict in this way</span>
<span class="sd"> &gt;&gt;&gt; df.replace({&#39;weapon&#39;: {&#39;Mjolnir&#39;: &#39;Stormbuster&#39;}})</span>
<span class="sd"> name weapon</span>
<span class="sd"> 0 Rescue Mark-45</span>
<span class="sd"> 1 Hawkeye Shield</span>
<span class="sd"> 2 Thor Stormbuster</span>
<span class="sd"> 3 Hulk Smash</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">method</span> <span class="o">!=</span> <span class="s2">&quot;pad&quot;</span><span class="p">:</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;The &#39;method&#39; keyword in DataFrame.replace is deprecated &quot;</span>
<span class="s2">&quot;and will be removed in a future version.&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;replace currently works only for method=&#39;pad&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">limit</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;The &#39;limit&#39; keyword in DataFrame.replace is deprecated &quot;</span>
<span class="s2">&quot;and will be removed in a future version.&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;replace currently works only when limit=None&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">regex</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">False</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;replace currently doesn&#39;t supports regex&quot;</span><span class="p">)</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">value</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="p">(</span><span class="nb">int</span><span class="p">,</span> <span class="nb">float</span><span class="p">,</span> <span class="nb">str</span><span class="p">,</span> <span class="nb">list</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Unsupported type </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">value</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">))</span>
<span class="k">if</span> <span class="n">to_replace</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">to_replace</span><span class="p">,</span> <span class="p">(</span><span class="nb">int</span><span class="p">,</span> <span class="nb">float</span><span class="p">,</span> <span class="nb">str</span><span class="p">,</span> <span class="nb">list</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span>
<span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Unsupported type </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">to_replace</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="p">(</span><span class="nb">list</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">))</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">to_replace</span><span class="p">,</span> <span class="p">(</span><span class="nb">list</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">)):</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">len</span><span class="p">(</span><span class="n">to_replace</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Length of to_replace and value must be same&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">to_replace</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span> <span class="ow">and</span> <span class="p">(</span>
<span class="n">value</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">or</span> <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">to_replace</span><span class="o">.</span><span class="n">values</span><span class="p">())</span>
<span class="p">):</span>
<span class="n">to_replace_dict</span> <span class="o">=</span> <span class="n">to_replace</span>
<span class="k">def</span> <span class="nf">op</span><span class="p">(</span><span class="n">psser</span><span class="p">:</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">:</span>
<span class="k">if</span> <span class="n">psser</span><span class="o">.</span><span class="n">name</span> <span class="ow">in</span> <span class="n">to_replace_dict</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psser</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span>
<span class="n">to_replace</span><span class="o">=</span><span class="n">to_replace_dict</span><span class="p">[</span><span class="n">psser</span><span class="o">.</span><span class="n">name</span><span class="p">],</span> <span class="n">value</span><span class="o">=</span><span class="n">value</span><span class="p">,</span> <span class="n">regex</span><span class="o">=</span><span class="n">regex</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psser</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="n">value</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;DataFrame.replace without &#39;value&#39; and with non-dict-like &#39;to_replace&#39; &quot;</span>
<span class="s2">&quot;is deprecated and will raise in a future version. &quot;</span>
<span class="s2">&quot;Explicitly specify the new values instead.&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">def</span> <span class="nf">op</span><span class="p">(</span><span class="n">psser</span><span class="p">:</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psser</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span>
<span class="n">to_replace</span><span class="o">=</span><span class="n">to_replace</span><span class="p">,</span> <span class="n">value</span><span class="o">=</span><span class="n">value</span><span class="p">,</span> <span class="n">regex</span><span class="o">=</span><span class="n">regex</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="n">op</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.clip"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.clip.html#pyspark.pandas.DataFrame.clip">[docs]</a> <span class="k">def</span> <span class="nf">clip</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">lower</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">float</span><span class="p">,</span> <span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> <span class="n">upper</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">float</span><span class="p">,</span> <span class="nb">int</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="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Trim values at input threshold(s).</span>
<span class="sd"> Assigns values outside boundary-to-boundary values.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> lower : float or int, default None</span>
<span class="sd"> Minimum threshold value. All values below this threshold will be set to it.</span>
<span class="sd"> upper : float or int, default None</span>
<span class="sd"> Maximum threshold value. All values above this threshold will be set to it.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame with the values outside the clip boundaries replaced.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame({&#39;A&#39;: [0, 2, 4]}).clip(1, 3)</span>
<span class="sd"> A</span>
<span class="sd"> 0 1</span>
<span class="sd"> 1 2</span>
<span class="sd"> 2 3</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> One difference between this implementation and pandas is that running</span>
<span class="sd"> pd.DataFrame({&#39;A&#39;: [&#39;a&#39;, &#39;b&#39;]}).clip(0, 1) will crash with &quot;TypeError: &#39;&lt;=&#39; not supported</span>
<span class="sd"> between instances of &#39;str&#39; and &#39;int&#39;&quot; while ps.DataFrame({&#39;A&#39;: [&#39;a&#39;, &#39;b&#39;]}).clip(0, 1)</span>
<span class="sd"> will output the original DataFrame, simply ignoring the incompatible types.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">lower</span><span class="p">)</span> <span class="ow">or</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">upper</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;List-like value are not supported for &#39;lower&#39; and &#39;upper&#39; at the &quot;</span> <span class="o">+</span> <span class="s2">&quot;moment&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">lower</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">upper</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">clip</span><span class="p">(</span><span class="n">lower</span><span class="o">=</span><span class="n">lower</span><span class="p">,</span> <span class="n">upper</span><span class="o">=</span><span class="n">upper</span><span class="p">))</span></div>
<div class="viewcode-block" id="DataFrame.head"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.head.html#pyspark.pandas.DataFrame.head">[docs]</a> <span class="k">def</span> <span class="nf">head</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="mi">5</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the first `n` rows.</span>
<span class="sd"> This function returns the first `n` rows for the object based</span>
<span class="sd"> on position. It is useful for quickly testing if your object</span>
<span class="sd"> has the right type of data in it.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> n : int, default 5</span>
<span class="sd"> Number of rows to select.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> obj_head : same type as caller</span>
<span class="sd"> The first `n` rows of the caller object.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;animal&#39;:[&#39;alligator&#39;, &#39;bee&#39;, &#39;falcon&#39;, &#39;lion&#39;,</span>
<span class="sd"> ... &#39;monkey&#39;, &#39;parrot&#39;, &#39;shark&#39;, &#39;whale&#39;, &#39;zebra&#39;]})</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> animal</span>
<span class="sd"> 0 alligator</span>
<span class="sd"> 1 bee</span>
<span class="sd"> 2 falcon</span>
<span class="sd"> 3 lion</span>
<span class="sd"> 4 monkey</span>
<span class="sd"> 5 parrot</span>
<span class="sd"> 6 shark</span>
<span class="sd"> 7 whale</span>
<span class="sd"> 8 zebra</span>
<span class="sd"> Viewing the first 5 lines</span>
<span class="sd"> &gt;&gt;&gt; df.head()</span>
<span class="sd"> animal</span>
<span class="sd"> 0 alligator</span>
<span class="sd"> 1 bee</span>
<span class="sd"> 2 falcon</span>
<span class="sd"> 3 lion</span>
<span class="sd"> 4 monkey</span>
<span class="sd"> Viewing the first `n` lines (three in this case)</span>
<span class="sd"> &gt;&gt;&gt; df.head(3)</span>
<span class="sd"> animal</span>
<span class="sd"> 0 alligator</span>
<span class="sd"> 1 bee</span>
<span class="sd"> 2 falcon</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">n</span> <span class="o">&lt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">n</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">+</span> <span class="n">n</span>
<span class="k">if</span> <span class="n">n</span> <span class="o">&lt;=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">)))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="k">if</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.ordered_head&quot;</span><span class="p">):</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">orderBy</span><span class="p">(</span><span class="n">NATURAL_ORDER_COLUMN_NAME</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">sdf</span><span class="o">.</span><span class="n">limit</span><span class="p">(</span><span class="n">n</span><span class="p">)))</span></div>
<div class="viewcode-block" id="DataFrame.last"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.last.html#pyspark.pandas.DataFrame.last">[docs]</a> <span class="k">def</span> <span class="nf">last</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">offset</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">DateOffset</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Select final periods of time series data based on a date offset.</span>
<span class="sd"> When having a DataFrame with dates as index, this function can</span>
<span class="sd"> select the last few rows based on a date offset.</span>
<span class="sd"> .. deprecated:: 4.0.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> offset : str or DateOffset</span>
<span class="sd"> The offset length of the data that will be selected. For instance,</span>
<span class="sd"> &#39;3D&#39; will display all the rows having their index within the last 3 days.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> A subset of the caller.</span>
<span class="sd"> Raises</span>
<span class="sd"> ------</span>
<span class="sd"> TypeError</span>
<span class="sd"> If the index is not a :class:`DatetimeIndex`</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; index = pd.date_range(&#39;2018-04-09&#39;, periods=4, freq=&#39;2D&#39;)</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;A&#39;: [1, 2, 3, 4]}, index=index)</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-09 1</span>
<span class="sd"> 2018-04-11 2</span>
<span class="sd"> 2018-04-13 3</span>
<span class="sd"> 2018-04-15 4</span>
<span class="sd"> Get the rows for the last 3 days:</span>
<span class="sd"> &gt;&gt;&gt; psdf.last(&#39;3D&#39;)</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-13 3</span>
<span class="sd"> 2018-04-15 4</span>
<span class="sd"> Notice the data for 3 last calendar days were returned, not the last</span>
<span class="sd"> 3 observed days in the dataset, and therefore data for 2018-04-11 was</span>
<span class="sd"> not returned.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;last is deprecated and will be removed in a future version. &quot;</span>
<span class="s2">&quot;Please create a mask and filter using `.loc` instead&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span><span class="p">,</span>
<span class="p">)</span>
<span class="c1"># Check index type should be format DateTime</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">DatetimeIndex</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;&#39;last&#39; only supports a DatetimeIndex&quot;</span><span class="p">)</span>
<span class="n">from_date</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span>
<span class="nb">int</span><span class="p">,</span>
<span class="n">cast</span><span class="p">(</span><span class="n">datetime</span><span class="o">.</span><span class="n">datetime</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">max</span><span class="p">())</span> <span class="o">-</span> <span class="n">cast</span><span class="p">(</span><span class="n">datetime</span><span class="o">.</span><span class="n">timedelta</span><span class="p">,</span> <span class="n">to_offset</span><span class="p">(</span><span class="n">offset</span><span class="p">)),</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">from_date</span><span class="p">:])</span></div>
<div class="viewcode-block" id="DataFrame.first"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.first.html#pyspark.pandas.DataFrame.first">[docs]</a> <span class="k">def</span> <span class="nf">first</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">offset</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">DateOffset</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Select first periods of time series data based on a date offset.</span>
<span class="sd"> When having a DataFrame with dates as index, this function can</span>
<span class="sd"> select the first few rows based on a date offset.</span>
<span class="sd"> .. deprecated:: 4.0.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> offset : str or DateOffset</span>
<span class="sd"> The offset length of the data that will be selected. For instance,</span>
<span class="sd"> &#39;3D&#39; will display all the rows having their index within the first 3 days.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> A subset of the caller.</span>
<span class="sd"> Raises</span>
<span class="sd"> ------</span>
<span class="sd"> TypeError</span>
<span class="sd"> If the index is not a :class:`DatetimeIndex`</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; index = pd.date_range(&#39;2018-04-09&#39;, periods=4, freq=&#39;2D&#39;)</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;A&#39;: [1, 2, 3, 4]}, index=index)</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-09 1</span>
<span class="sd"> 2018-04-11 2</span>
<span class="sd"> 2018-04-13 3</span>
<span class="sd"> 2018-04-15 4</span>
<span class="sd"> Get the rows for the last 3 days:</span>
<span class="sd"> &gt;&gt;&gt; psdf.first(&#39;3D&#39;)</span>
<span class="sd"> A</span>
<span class="sd"> 2018-04-09 1</span>
<span class="sd"> 2018-04-11 2</span>
<span class="sd"> Notice the data for 3 first calendar days were returned, not the first</span>
<span class="sd"> 3 observed days in the dataset, and therefore data for 2018-04-13 was</span>
<span class="sd"> not returned.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;first is deprecated and will be removed in a future version. &quot;</span>
<span class="s2">&quot;Please create a mask and filter using `.loc` instead&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span><span class="p">,</span>
<span class="p">)</span>
<span class="c1"># Check index type should be format DatetimeIndex</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">DatetimeIndex</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;&#39;first&#39; only supports a DatetimeIndex&quot;</span><span class="p">)</span>
<span class="n">to_date</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span>
<span class="nb">int</span><span class="p">,</span>
<span class="n">cast</span><span class="p">(</span><span class="n">datetime</span><span class="o">.</span><span class="n">datetime</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">min</span><span class="p">())</span> <span class="o">+</span> <span class="n">cast</span><span class="p">(</span><span class="n">datetime</span><span class="o">.</span><span class="n">timedelta</span><span class="p">,</span> <span class="n">to_offset</span><span class="p">(</span><span class="n">offset</span><span class="p">)),</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[:</span><span class="n">to_date</span><span class="p">])</span></div>
<div class="viewcode-block" id="DataFrame.pivot_table"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.pivot_table.html#pyspark.pandas.DataFrame.pivot_table">[docs]</a> <span class="k">def</span> <span class="nf">pivot_table</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">values</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Name</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">aggfunc</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dict</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="o">=</span> <span class="s2">&quot;mean&quot;</span><span class="p">,</span>
<span class="n">fill_value</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Any</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Create a spreadsheet-style pivot table as a DataFrame. The levels in</span>
<span class="sd"> the pivot table will be stored in MultiIndex objects (hierarchical</span>
<span class="sd"> indexes) on the index and columns of the result DataFrame.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> values : column to aggregate.</span>
<span class="sd"> They should be either a list less than three or a string.</span>
<span class="sd"> index : column (string) or list of columns</span>
<span class="sd"> If an array is passed, it must be the same length as the data.</span>
<span class="sd"> The list should contain string.</span>
<span class="sd"> columns : column</span>
<span class="sd"> Columns used in the pivot operation. Only one column is supported and</span>
<span class="sd"> it should be a string.</span>
<span class="sd"> aggfunc : function (string), dict, default mean</span>
<span class="sd"> If dict is passed, the key is column to aggregate and value</span>
<span class="sd"> is function or list of functions.</span>
<span class="sd"> fill_value : scalar, default None</span>
<span class="sd"> Value to replace missing values with.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> table : DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&quot;A&quot;: [&quot;foo&quot;, &quot;foo&quot;, &quot;foo&quot;, &quot;foo&quot;, &quot;foo&quot;,</span>
<span class="sd"> ... &quot;bar&quot;, &quot;bar&quot;, &quot;bar&quot;, &quot;bar&quot;],</span>
<span class="sd"> ... &quot;B&quot;: [&quot;one&quot;, &quot;one&quot;, &quot;one&quot;, &quot;two&quot;, &quot;two&quot;,</span>
<span class="sd"> ... &quot;one&quot;, &quot;one&quot;, &quot;two&quot;, &quot;two&quot;],</span>
<span class="sd"> ... &quot;C&quot;: [&quot;small&quot;, &quot;large&quot;, &quot;large&quot;, &quot;small&quot;,</span>
<span class="sd"> ... &quot;small&quot;, &quot;large&quot;, &quot;small&quot;, &quot;small&quot;,</span>
<span class="sd"> ... &quot;large&quot;],</span>
<span class="sd"> ... &quot;D&quot;: [1, 2, 2, 3, 3, 4, 5, 6, 7],</span>
<span class="sd"> ... &quot;E&quot;: [2, 4, 5, 5, 6, 6, 8, 9, 9]},</span>
<span class="sd"> ... columns=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;, &#39;D&#39;, &#39;E&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C D E</span>
<span class="sd"> 0 foo one small 1 2</span>
<span class="sd"> 1 foo one large 2 4</span>
<span class="sd"> 2 foo one large 2 5</span>
<span class="sd"> 3 foo two small 3 5</span>
<span class="sd"> 4 foo two small 3 6</span>
<span class="sd"> 5 bar one large 4 6</span>
<span class="sd"> 6 bar one small 5 8</span>
<span class="sd"> 7 bar two small 6 9</span>
<span class="sd"> 8 bar two large 7 9</span>
<span class="sd"> This first example aggregates values by taking the sum.</span>
<span class="sd"> &gt;&gt;&gt; table = df.pivot_table(values=&#39;D&#39;, index=[&#39;A&#39;, &#39;B&#39;],</span>
<span class="sd"> ... columns=&#39;C&#39;, aggfunc=&#39;sum&#39;)</span>
<span class="sd"> &gt;&gt;&gt; table.sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> C large small</span>
<span class="sd"> A B</span>
<span class="sd"> bar one 4.0 5</span>
<span class="sd"> two 7.0 6</span>
<span class="sd"> foo one 4.0 1</span>
<span class="sd"> two NaN 6</span>
<span class="sd"> We can also fill missing values using the `fill_value` parameter.</span>
<span class="sd"> &gt;&gt;&gt; table = df.pivot_table(values=&#39;D&#39;, index=[&#39;A&#39;, &#39;B&#39;],</span>
<span class="sd"> ... columns=&#39;C&#39;, aggfunc=&#39;sum&#39;, fill_value=0)</span>
<span class="sd"> &gt;&gt;&gt; table.sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> C large small</span>
<span class="sd"> A B</span>
<span class="sd"> bar one 4 5</span>
<span class="sd"> two 7 6</span>
<span class="sd"> foo one 4 1</span>
<span class="sd"> two 0 6</span>
<span class="sd"> We can also calculate multiple types of aggregations for any given</span>
<span class="sd"> value column.</span>
<span class="sd"> &gt;&gt;&gt; table = df.pivot_table(values=[&#39;D&#39;], index =[&#39;C&#39;],</span>
<span class="sd"> ... columns=&quot;A&quot;, aggfunc={&#39;D&#39;: &#39;mean&#39;})</span>
<span class="sd"> &gt;&gt;&gt; table.sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> D</span>
<span class="sd"> A bar foo</span>
<span class="sd"> C</span>
<span class="sd"> large 5.5 2.000000</span>
<span class="sd"> small 5.5 2.333333</span>
<span class="sd"> The next example aggregates on multiple values.</span>
<span class="sd"> &gt;&gt;&gt; table = df.pivot_table(index=[&#39;C&#39;], columns=&quot;A&quot;, values=[&#39;D&#39;, &#39;E&#39;],</span>
<span class="sd"> ... aggfunc={&#39;D&#39;: &#39;mean&#39;, &#39;E&#39;: &#39;sum&#39;})</span>
<span class="sd"> &gt;&gt;&gt; table.sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> D E</span>
<span class="sd"> A bar foo bar foo</span>
<span class="sd"> C</span>
<span class="sd"> large 5.5 2.000000 15 9</span>
<span class="sd"> small 5.5 2.333333 17 13</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">columns</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;columns should be one column name.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">values</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="p">(</span>
<span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">all</span><span class="p">(</span><span class="n">is_name_like_value</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">values</span><span class="p">)</span>
<span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;values should be one column or list of columns.&quot;</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">aggfunc</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="ow">and</span> <span class="p">(</span>
<span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">aggfunc</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span>
<span class="ow">or</span> <span class="ow">not</span> <span class="nb">all</span><span class="p">(</span>
<span class="n">is_name_like_value</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span> <span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">aggfunc</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">)</span>
<span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;aggfunc must be a dict mapping from column name &quot;</span>
<span class="s2">&quot;to aggregate functions (string).&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">aggfunc</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span> <span class="ow">and</span> <span class="n">index</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span>
<span class="s2">&quot;pivot_table doesn&#39;t support aggfunc&quot;</span> <span class="s2">&quot; as dict and without index.&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="ow">and</span> <span class="n">index</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;values can&#39;t be a list without index.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">columns</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Wrong columns </span><span class="si">{}</span><span class="s2">.&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">columns</span><span class="p">)))</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">columns</span><span class="p">):</span>
<span class="n">columns</span> <span class="o">=</span> <span class="p">(</span><span class="n">columns</span><span class="p">,)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="n">values</span> <span class="o">=</span> <span class="p">[</span><span class="n">col</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">col</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">col</span><span class="p">,)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">values</span><span class="p">]</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">all</span><span class="p">(</span>
<span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">col</span><span class="p">),</span> <span class="n">NumericType</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">values</span>
<span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;values should be a numeric type.&quot;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">values</span> <span class="o">=</span> <span class="n">values</span> <span class="k">if</span> <span class="n">is_name_like_tuple</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="n">values</span><span class="p">,)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">values</span><span class="p">),</span> <span class="n">NumericType</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;values should be a numeric type.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">aggfunc</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="n">agg_cols</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">expr</span><span class="p">(</span>
<span class="s2">&quot;</span><span class="si">{1}</span><span class="s2">(`</span><span class="si">{0}</span><span class="s2">`) as `</span><span class="si">{0}</span><span class="s2">`&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">value</span><span class="p">),</span> <span class="n">aggfunc</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">values</span>
<span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">agg_cols</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">expr</span><span class="p">(</span>
<span class="s2">&quot;</span><span class="si">{1}</span><span class="s2">(`</span><span class="si">{0}</span><span class="s2">`) as `</span><span class="si">{0}</span><span class="s2">`&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">values</span><span class="p">),</span> <span class="n">aggfunc</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="p">]</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">aggfunc</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="n">aggfunc</span> <span class="o">=</span> <span class="p">{</span>
<span class="n">key</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">key</span><span class="p">,):</span> <span class="n">value</span> <span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">aggfunc</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">}</span>
<span class="n">agg_cols</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">expr</span><span class="p">(</span>
<span class="s2">&quot;</span><span class="si">{1}</span><span class="s2">(`</span><span class="si">{0}</span><span class="s2">`) as `</span><span class="si">{0}</span><span class="s2">`&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">key</span><span class="p">),</span> <span class="n">value</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">aggfunc</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="p">]</span>
<span class="n">agg_columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">key</span> <span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">_</span> <span class="ow">in</span> <span class="n">aggfunc</span><span class="o">.</span><span class="n">items</span><span class="p">()]</span>
<span class="k">if</span> <span class="nb">set</span><span class="p">(</span><span class="n">agg_columns</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">set</span><span class="p">(</span><span class="n">values</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Columns in aggfunc must be the same as values.&quot;</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">sdf</span><span class="o">.</span><span class="n">groupBy</span><span class="p">()</span>
<span class="o">.</span><span class="n">pivot</span><span class="p">(</span><span class="n">pivot_col</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">columns</span><span class="p">))</span>
<span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="o">*</span><span class="n">agg_cols</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="n">index</span> <span class="o">=</span> <span class="p">[</span><span class="n">label</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">label</span><span class="p">,)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">index</span><span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">sdf</span><span class="o">.</span><span class="n">groupBy</span><span class="p">([</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">index</span><span class="p">])</span>
<span class="o">.</span><span class="n">pivot</span><span class="p">(</span><span class="n">pivot_col</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">columns</span><span class="p">))</span>
<span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="o">*</span><span class="n">agg_cols</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;index should be a None or a list of columns.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">fill_value</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">fill_value</span><span class="p">,</span> <span class="p">(</span><span class="nb">int</span><span class="p">,</span> <span class="nb">float</span><span class="p">)):</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">fillna</span><span class="p">(</span><span class="n">fill_value</span><span class="p">)</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">index_columns</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">index</span><span class="p">]</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">field_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">index</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="n">data_columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">column</span> <span class="k">for</span> <span class="n">column</span> <span class="ow">in</span> <span class="n">sdf</span><span class="o">.</span><span class="n">columns</span> <span class="k">if</span> <span class="n">column</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">index_columns</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="c1"># If we have two values, Spark will return column&#39;s name</span>
<span class="c1"># in this format: column_values, where column contains</span>
<span class="c1"># their values in the DataFrame and values is</span>
<span class="c1"># the column list passed to the pivot_table().</span>
<span class="c1"># E.g. if column is b and values is [&#39;b&#39;,&#39;e&#39;],</span>
<span class="c1"># then [&#39;2_b&#39;, &#39;2_e&#39;, &#39;3_b&#39;, &#39;3_e&#39;].</span>
<span class="c1"># We sort the columns of Spark DataFrame by values.</span>
<span class="n">data_columns</span><span class="o">.</span><span class="n">sort</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="o">.</span><span class="n">split</span><span class="p">(</span><span class="s2">&quot;_&quot;</span><span class="p">,</span> <span class="mi">1</span><span class="p">)[</span><span class="mi">1</span><span class="p">])</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">index_columns</span> <span class="o">+</span> <span class="n">data_columns</span><span class="p">)</span>
<span class="n">column_name_to_index</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">(</span>
<span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="nb">tuple</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="n">column_name_to_index</span><span class="p">[</span><span class="n">name</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s2">&quot;_&quot;</span><span class="p">)[</span><span class="mi">1</span><span class="p">]])</span> <span class="o">+</span> <span class="p">[</span><span class="n">name</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="s2">&quot;_&quot;</span><span class="p">)[</span><span class="mi">0</span><span class="p">]])</span>
<span class="k">for</span> <span class="n">name</span> <span class="ow">in</span> <span class="n">data_columns</span>
<span class="p">]</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="p">(</span>
<span class="p">[</span><span class="n">cast</span><span class="p">(</span><span class="n">Optional</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="o">*</span> <span class="n">column_labels_level</span><span class="p">(</span><span class="n">values</span><span class="p">)</span>
<span class="p">)</span> <span class="o">+</span> <span class="p">[</span><span class="n">columns</span><span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">index_columns</span><span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">data_columns</span><span class="p">],</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="n">column_label_names</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span><span class="nb">tuple</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="n">values</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> <span class="o">+</span> <span class="p">[</span><span class="n">column</span><span class="p">])</span> <span class="k">for</span> <span class="n">column</span> <span class="ow">in</span> <span class="n">data_columns</span><span class="p">]</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="p">([</span><span class="n">cast</span><span class="p">(</span><span class="n">Optional</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="o">*</span> <span class="nb">len</span><span class="p">(</span><span class="n">values</span><span class="p">[</span><span class="mi">0</span><span class="p">]))</span> <span class="o">+</span> <span class="p">[</span><span class="n">columns</span><span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">index_columns</span><span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">data_columns</span><span class="p">],</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="n">column_label_names</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">index_columns</span><span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span><span class="p">,</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="p">[</span><span class="n">columns</span><span class="p">],</span>
<span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">index_values</span> <span class="o">=</span> <span class="n">values</span>
<span class="n">index_map</span><span class="p">:</span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Label</span><span class="p">]]</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">index_value</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">index_values</span><span class="p">):</span>
<span class="n">colname</span> <span class="o">=</span> <span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="n">colname</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">index_value</span><span class="p">))</span>
<span class="n">index_map</span><span class="p">[</span><span class="n">colname</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">index_map</span><span class="o">.</span><span class="n">keys</span><span class="p">()],</span>
<span class="n">index_names</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_map</span><span class="o">.</span><span class="n">values</span><span class="p">()),</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="p">[</span><span class="n">columns</span><span class="p">],</span>
<span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="n">psdf_columns</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">columns</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">psdf_columns</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">):</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">columns</span> <span class="o">=</span> <span class="n">psdf_columns</span><span class="o">.</span><span class="n">set_levels</span><span class="p">(</span>
<span class="n">psdf_columns</span><span class="o">.</span><span class="n">levels</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span> <span class="c1"># type: ignore[index]</span>
<span class="n">spark_type_to_pandas_dtype</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">)</span>
<span class="p">),</span>
<span class="n">level</span><span class="o">=-</span><span class="mi">1</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">columns</span> <span class="o">=</span> <span class="n">psdf_columns</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span>
<span class="n">spark_type_to_pandas_dtype</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.pivot"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.pivot.html#pyspark.pandas.DataFrame.pivot">[docs]</a> <span class="k">def</span> <span class="nf">pivot</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Name</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Name</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">values</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Name</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return reshaped DataFrame organized by given index / column values.</span>
<span class="sd"> Reshape data (produce a &quot;pivot&quot; table) based on column values. Uses</span>
<span class="sd"> unique values from specified `index` / `columns` to form axes of the</span>
<span class="sd"> resulting DataFrame. This function does not support data</span>
<span class="sd"> aggregation.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> index : string, optional</span>
<span class="sd"> Column to use to make new frame&#39;s index. If None, uses</span>
<span class="sd"> existing index.</span>
<span class="sd"> columns : string</span>
<span class="sd"> Column to use to make new frame&#39;s columns.</span>
<span class="sd"> values : string, object or a list of the previous</span>
<span class="sd"> Column(s) to use for populating new frame&#39;s values.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Returns reshaped DataFrame.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.pivot_table : Generalization of pivot that can handle</span>
<span class="sd"> duplicate values for one index/column pair.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;foo&#39;: [&#39;one&#39;, &#39;one&#39;, &#39;one&#39;, &#39;two&#39;, &#39;two&#39;,</span>
<span class="sd"> ... &#39;two&#39;],</span>
<span class="sd"> ... &#39;bar&#39;: [&#39;A&#39;, &#39;B&#39;, &#39;C&#39;, &#39;A&#39;, &#39;B&#39;, &#39;C&#39;],</span>
<span class="sd"> ... &#39;baz&#39;: [1, 2, 3, 4, 5, 6],</span>
<span class="sd"> ... &#39;zoo&#39;: [&#39;x&#39;, &#39;y&#39;, &#39;z&#39;, &#39;q&#39;, &#39;w&#39;, &#39;t&#39;]},</span>
<span class="sd"> ... columns=[&#39;foo&#39;, &#39;bar&#39;, &#39;baz&#39;, &#39;zoo&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> foo bar baz zoo</span>
<span class="sd"> 0 one A 1 x</span>
<span class="sd"> 1 one B 2 y</span>
<span class="sd"> 2 one C 3 z</span>
<span class="sd"> 3 two A 4 q</span>
<span class="sd"> 4 two B 5 w</span>
<span class="sd"> 5 two C 6 t</span>
<span class="sd"> &gt;&gt;&gt; df.pivot(index=&#39;foo&#39;, columns=&#39;bar&#39;, values=&#39;baz&#39;).sort_index()</span>
<span class="sd"> ... # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> bar A B C</span>
<span class="sd"> foo</span>
<span class="sd"> one 1 2 3</span>
<span class="sd"> two 4 5 6</span>
<span class="sd"> &gt;&gt;&gt; df.pivot(columns=&#39;bar&#39;, values=&#39;baz&#39;).sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> bar A B C</span>
<span class="sd"> 0 1.0 NaN NaN</span>
<span class="sd"> 1 NaN 2.0 NaN</span>
<span class="sd"> 2 NaN NaN 3.0</span>
<span class="sd"> 3 4.0 NaN NaN</span>
<span class="sd"> 4 NaN 5.0 NaN</span>
<span class="sd"> 5 NaN NaN 6.0</span>
<span class="sd"> Notice that, unlike pandas raises an ValueError when duplicated values are found.</span>
<span class="sd"> Pandas-on-Spark&#39;s pivot still works with its first value it meets during operation because</span>
<span class="sd"> pivot is an expensive operation, and it is preferred to permissively execute over failing</span>
<span class="sd"> fast when processing large data.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&quot;foo&quot;: [&#39;one&#39;, &#39;one&#39;, &#39;two&#39;, &#39;two&#39;],</span>
<span class="sd"> ... &quot;bar&quot;: [&#39;A&#39;, &#39;A&#39;, &#39;B&#39;, &#39;C&#39;],</span>
<span class="sd"> ... &quot;baz&quot;: [1, 2, 3, 4]}, columns=[&#39;foo&#39;, &#39;bar&#39;, &#39;baz&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> foo bar baz</span>
<span class="sd"> 0 one A 1</span>
<span class="sd"> 1 one A 2</span>
<span class="sd"> 2 two B 3</span>
<span class="sd"> 3 two C 4</span>
<span class="sd"> &gt;&gt;&gt; df.pivot(index=&#39;foo&#39;, columns=&#39;bar&#39;, values=&#39;baz&#39;).sort_index()</span>
<span class="sd"> ... # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> bar A B C</span>
<span class="sd"> foo</span>
<span class="sd"> one 1.0 NaN NaN</span>
<span class="sd"> two NaN 3.0 4.0</span>
<span class="sd"> It also supports multi-index and multi-index column.</span>
<span class="sd"> &gt;&gt;&gt; df.columns = pd.MultiIndex.from_tuples([(&#39;a&#39;, &#39;foo&#39;), (&#39;a&#39;, &#39;bar&#39;), (&#39;b&#39;, &#39;baz&#39;)])</span>
<span class="sd"> &gt;&gt;&gt; df = df.set_index((&#39;a&#39;, &#39;bar&#39;), append=True)</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> a b</span>
<span class="sd"> foo baz</span>
<span class="sd"> (a, bar)</span>
<span class="sd"> 0 A one 1</span>
<span class="sd"> 1 A one 2</span>
<span class="sd"> 2 B two 3</span>
<span class="sd"> 3 C two 4</span>
<span class="sd"> &gt;&gt;&gt; df.pivot(columns=(&#39;a&#39;, &#39;foo&#39;), values=(&#39;b&#39;, &#39;baz&#39;)).sort_index()</span>
<span class="sd"> ... # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> (&#39;a&#39;, &#39;foo&#39;) one two</span>
<span class="sd"> (a, bar)</span>
<span class="sd"> 0 A 1.0 NaN</span>
<span class="sd"> 1 A 2.0 NaN</span>
<span class="sd"> 2 B NaN 3.0</span>
<span class="sd"> 3 C NaN 4.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">columns</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;columns should be set.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">values</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;values should be set.&quot;</span><span class="p">)</span>
<span class="n">should_use_existing_index</span> <span class="o">=</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="k">if</span> <span class="n">should_use_existing_index</span><span class="p">:</span>
<span class="n">df</span> <span class="o">=</span> <span class="bp">self</span>
<span class="n">index_labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">index</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># The index after `reset_index()` will never be used, so use &quot;distributed&quot; index</span>
<span class="c1"># as a dummy to avoid overhead.</span>
<span class="k">with</span> <span class="n">option_context</span><span class="p">(</span><span class="s2">&quot;compute.default_index_type&quot;</span><span class="p">,</span> <span class="s2">&quot;distributed&quot;</span><span class="p">):</span>
<span class="n">df</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">reset_index</span><span class="p">()</span>
<span class="n">index_labels</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">[:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">]</span>
<span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">pivot_table</span><span class="p">(</span><span class="n">index</span><span class="o">=</span><span class="n">index_labels</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">,</span> <span class="n">values</span><span class="o">=</span><span class="n">values</span><span class="p">,</span> <span class="n">aggfunc</span><span class="o">=</span><span class="s2">&quot;first&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">should_use_existing_index</span><span class="p">:</span>
<span class="k">return</span> <span class="n">df</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">index_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span></div>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">columns</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;The column labels of the DataFrame.&quot;&quot;&quot;</span>
<span class="n">names</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">name</span> <span class="k">if</span> <span class="n">name</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="nb">len</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</span> <span class="k">else</span> <span class="n">name</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">for</span> <span class="n">name</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span>
<span class="p">]</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">columns</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="o">.</span><span class="n">from_tuples</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">names</span><span class="o">=</span><span class="n">names</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">columns</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">([</span><span class="n">label</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">],</span> <span class="n">name</span><span class="o">=</span><span class="n">names</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span>
<span class="k">return</span> <span class="n">columns</span>
<span class="nd">@columns</span><span class="o">.</span><span class="n">setter</span>
<span class="k">def</span> <span class="nf">columns</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">columns</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]])</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">):</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="n">columns</span><span class="o">.</span><span class="n">tolist</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">col</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">col</span><span class="p">,</span> <span class="n">allow_none</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">col</span><span class="p">,)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">columns</span>
<span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">len</span><span class="p">(</span><span class="n">column_labels</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Length mismatch: Expected axis has </span><span class="si">{}</span><span class="s2"> elements, &quot;</span>
<span class="s2">&quot;new values have </span><span class="si">{}</span><span class="s2"> elements&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">),</span> <span class="nb">len</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">column_label_names</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">):</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">name</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="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">columns</span><span class="o">.</span><span class="n">names</span>
<span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">pssers</span> <span class="o">=</span> <span class="p">[</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">name</span><span class="p">)</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">column_labels</span><span class="p">)</span>
<span class="p">]</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">pssers</span><span class="p">,</span> <span class="n">column_label_names</span><span class="o">=</span><span class="n">column_label_names</span><span class="p">)</span>
<span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">dtypes</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;Return the dtypes in the DataFrame.</span>
<span class="sd"> This returns a Series with the data type of each column. The result&#39;s index is the original</span>
<span class="sd"> DataFrame&#39;s columns. Columns with mixed types are stored with the object dtype.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> pd.Series</span>
<span class="sd"> The data type of each column.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: list(&#39;abc&#39;),</span>
<span class="sd"> ... &#39;b&#39;: list(range(1, 4)),</span>
<span class="sd"> ... &#39;c&#39;: np.arange(3, 6).astype(&#39;i1&#39;),</span>
<span class="sd"> ... &#39;d&#39;: np.arange(4.0, 7.0, dtype=&#39;float64&#39;),</span>
<span class="sd"> ... &#39;e&#39;: [True, False, True],</span>
<span class="sd"> ... &#39;f&#39;: pd.date_range(&#39;20130101&#39;, periods=3)},</span>
<span class="sd"> ... columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;, &#39;f&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.dtypes</span>
<span class="sd"> a object</span>
<span class="sd"> b int64</span>
<span class="sd"> c int8</span>
<span class="sd"> d float64</span>
<span class="sd"> e bool</span>
<span class="sd"> f datetime64[ns]</span>
<span class="sd"> dtype: object</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">(</span>
<span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">dtype</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">],</span>
<span class="n">index</span><span class="o">=</span><span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span>
<span class="p">[</span><span class="n">label</span> <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</span> <span class="k">else</span> <span class="n">label</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">]</span>
<span class="p">),</span>
<span class="p">)</span>
<div class="viewcode-block" id="DataFrame.select_dtypes"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.select_dtypes.html#pyspark.pandas.DataFrame.select_dtypes">[docs]</a> <span class="k">def</span> <span class="nf">select_dtypes</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">include</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">exclude</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a subset of the DataFrame&#39;s columns based on the column dtypes.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> include, exclude : scalar or list-like</span>
<span class="sd"> A selection of dtypes or strings to be included/excluded. At least</span>
<span class="sd"> one of these parameters must be supplied. It also takes Spark SQL</span>
<span class="sd"> DDL type strings, for instance, &#39;string&#39; and &#39;date&#39;.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> The subset of the frame including the dtypes in ``include`` and</span>
<span class="sd"> excluding the dtypes in ``exclude``.</span>
<span class="sd"> Raises</span>
<span class="sd"> ------</span>
<span class="sd"> ValueError</span>
<span class="sd"> * If both of ``include`` and ``exclude`` are empty</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2] * 3,</span>
<span class="sd"> ... &#39;b&#39;: [True, False] * 3,</span>
<span class="sd"> ... &#39;c&#39;: [1.0, 2.0] * 3})</span>
<span class="sd"> &gt;&gt;&gt; df.select_dtypes()</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: at least one of include or exclude must be nonempty</span>
<span class="sd"> * If ``include`` and ``exclude`` have overlapping elements</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2] * 3,</span>
<span class="sd"> ... &#39;b&#39;: [True, False] * 3,</span>
<span class="sd"> ... &#39;c&#39;: [1.0, 2.0] * 3})</span>
<span class="sd"> &gt;&gt;&gt; df.select_dtypes(include=&#39;a&#39;, exclude=&#39;a&#39;)</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: include and exclude overlap on {&#39;a&#39;}</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> * To select datetimes, use ``np.datetime64``, ``&#39;datetime&#39;`` or</span>
<span class="sd"> ``&#39;datetime64&#39;``</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2] * 3,</span>
<span class="sd"> ... &#39;b&#39;: [True, False] * 3,</span>
<span class="sd"> ... &#39;c&#39;: [1.0, 2.0] * 3,</span>
<span class="sd"> ... &#39;d&#39;: [&#39;a&#39;, &#39;b&#39;] * 3}, columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> a b c d</span>
<span class="sd"> 0 1 True 1.0 a</span>
<span class="sd"> 1 2 False 2.0 b</span>
<span class="sd"> 2 1 True 1.0 a</span>
<span class="sd"> 3 2 False 2.0 b</span>
<span class="sd"> 4 1 True 1.0 a</span>
<span class="sd"> 5 2 False 2.0 b</span>
<span class="sd"> &gt;&gt;&gt; df.select_dtypes(include=&#39;bool&#39;)</span>
<span class="sd"> b</span>
<span class="sd"> 0 True</span>
<span class="sd"> 1 False</span>
<span class="sd"> 2 True</span>
<span class="sd"> 3 False</span>
<span class="sd"> 4 True</span>
<span class="sd"> 5 False</span>
<span class="sd"> &gt;&gt;&gt; df.select_dtypes(include=[&#39;float64&#39;], exclude=[&#39;int&#39;])</span>
<span class="sd"> c</span>
<span class="sd"> 0 1.0</span>
<span class="sd"> 1 2.0</span>
<span class="sd"> 2 1.0</span>
<span class="sd"> 3 2.0</span>
<span class="sd"> 4 1.0</span>
<span class="sd"> 5 2.0</span>
<span class="sd"> &gt;&gt;&gt; df.select_dtypes(include=[&#39;int&#39;], exclude=[&#39;float64&#39;])</span>
<span class="sd"> a</span>
<span class="sd"> 0 1</span>
<span class="sd"> 1 2</span>
<span class="sd"> 2 1</span>
<span class="sd"> 3 2</span>
<span class="sd"> 4 1</span>
<span class="sd"> 5 2</span>
<span class="sd"> &gt;&gt;&gt; df.select_dtypes(exclude=[&#39;int&#39;])</span>
<span class="sd"> b c d</span>
<span class="sd"> 0 True 1.0 a</span>
<span class="sd"> 1 False 2.0 b</span>
<span class="sd"> 2 True 1.0 a</span>
<span class="sd"> 3 False 2.0 b</span>
<span class="sd"> 4 True 1.0 a</span>
<span class="sd"> 5 False 2.0 b</span>
<span class="sd"> Spark SQL DDL type strings can be used as well.</span>
<span class="sd"> &gt;&gt;&gt; df.select_dtypes(exclude=[&#39;string&#39;])</span>
<span class="sd"> a b c</span>
<span class="sd"> 0 1 True 1.0</span>
<span class="sd"> 1 2 False 2.0</span>
<span class="sd"> 2 1 True 1.0</span>
<span class="sd"> 3 2 False 2.0</span>
<span class="sd"> 4 1 True 1.0</span>
<span class="sd"> 5 2 False 2.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.types</span> <span class="kn">import</span> <span class="n">_parse_datatype_string</span>
<span class="n">include_list</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">include</span><span class="p">):</span>
<span class="n">include_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">cast</span><span class="p">(</span><span class="nb">str</span><span class="p">,</span> <span class="n">include</span><span class="p">)]</span> <span class="k">if</span> <span class="n">include</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">else</span> <span class="p">[]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">include_list</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">include</span><span class="p">)</span>
<span class="n">exclude_list</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">exclude</span><span class="p">):</span>
<span class="n">exclude_list</span> <span class="o">=</span> <span class="p">[</span><span class="n">cast</span><span class="p">(</span><span class="nb">str</span><span class="p">,</span> <span class="n">exclude</span><span class="p">)]</span> <span class="k">if</span> <span class="n">exclude</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">else</span> <span class="p">[]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">exclude_list</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">exclude</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">any</span><span class="p">((</span><span class="n">include_list</span><span class="p">,</span> <span class="n">exclude_list</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;at least one of include or exclude must be &quot;</span> <span class="s2">&quot;nonempty&quot;</span><span class="p">)</span>
<span class="c1"># can&#39;t both include AND exclude!</span>
<span class="k">if</span> <span class="nb">set</span><span class="p">(</span><span class="n">include_list</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">exclude_list</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;include and exclude overlap on </span><span class="si">{inc_ex}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">inc_ex</span><span class="o">=</span><span class="nb">set</span><span class="p">(</span><span class="n">include_list</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">exclude_list</span><span class="p">))</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="c1"># Handle Spark types</span>
<span class="n">include_spark_type</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">inc</span> <span class="ow">in</span> <span class="n">include_list</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">include_spark_type</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">_parse_datatype_string</span><span class="p">(</span><span class="n">inc</span><span class="p">))</span>
<span class="k">except</span> <span class="ne">BaseException</span><span class="p">:</span>
<span class="k">pass</span>
<span class="n">exclude_spark_type</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">exc</span> <span class="ow">in</span> <span class="n">exclude_list</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">exclude_spark_type</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">_parse_datatype_string</span><span class="p">(</span><span class="n">exc</span><span class="p">))</span>
<span class="k">except</span> <span class="ne">BaseException</span><span class="p">:</span>
<span class="k">pass</span>
<span class="c1"># Handle pandas types</span>
<span class="n">include_numpy_type</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">inc</span> <span class="ow">in</span> <span class="n">include_list</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">include_numpy_type</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">infer_dtype_from_object</span><span class="p">(</span><span class="n">inc</span><span class="p">))</span>
<span class="k">except</span> <span class="ne">BaseException</span><span class="p">:</span>
<span class="k">pass</span>
<span class="n">exclude_numpy_type</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">exc</span> <span class="ow">in</span> <span class="n">exclude_list</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">exclude_numpy_type</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">infer_dtype_from_object</span><span class="p">(</span><span class="n">exc</span><span class="p">))</span>
<span class="k">except</span> <span class="ne">BaseException</span><span class="p">:</span>
<span class="k">pass</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">include_list</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">should_include</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">infer_dtype_from_object</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">dtype</span><span class="o">.</span><span class="n">name</span><span class="p">)</span> <span class="ow">in</span> <span class="n">include_numpy_type</span>
<span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="ow">in</span> <span class="n">include_spark_type</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">should_include</span> <span class="o">=</span> <span class="ow">not</span> <span class="p">(</span>
<span class="n">infer_dtype_from_object</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">dtype</span><span class="o">.</span><span class="n">name</span><span class="p">)</span> <span class="ow">in</span> <span class="n">exclude_numpy_type</span>
<span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="ow">in</span> <span class="n">exclude_spark_type</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">should_include</span><span class="p">:</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">([</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">])</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.droplevel"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.droplevel.html#pyspark.pandas.DataFrame.droplevel">[docs]</a> <span class="k">def</span> <span class="nf">droplevel</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">level</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">]]],</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return DataFrame with requested index / column level(s) removed.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> level: int, str, or list-like</span>
<span class="sd"> If a string is given, must be the name of a level If list-like, elements must</span>
<span class="sd"> be names or positional indexes of levels.</span>
<span class="sd"> axis: {0 or ‘index’, 1 or ‘columns’}, default 0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame with requested index / column level(s) removed.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(</span>
<span class="sd"> ... [[3, 4], [7, 8], [11, 12]],</span>
<span class="sd"> ... index=pd.MultiIndex.from_tuples([(1, 2), (5, 6), (9, 10)], names=[&quot;a&quot;, &quot;b&quot;]),</span>
<span class="sd"> ... )</span>
<span class="sd"> &gt;&gt;&gt; df.columns = pd.MultiIndex.from_tuples([</span>
<span class="sd"> ... (&#39;c&#39;, &#39;e&#39;), (&#39;d&#39;, &#39;f&#39;)</span>
<span class="sd"> ... ], names=[&#39;level_1&#39;, &#39;level_2&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> level_1 c d</span>
<span class="sd"> level_2 e f</span>
<span class="sd"> a b</span>
<span class="sd"> 1 2 3 4</span>
<span class="sd"> 5 6 7 8</span>
<span class="sd"> 9 10 11 12</span>
<span class="sd"> &gt;&gt;&gt; df.droplevel(&#39;a&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> level_1 c d</span>
<span class="sd"> level_2 e f</span>
<span class="sd"> b</span>
<span class="sd"> 2 3 4</span>
<span class="sd"> 6 7 8</span>
<span class="sd"> 10 11 12</span>
<span class="sd"> &gt;&gt;&gt; df.droplevel(&#39;level_2&#39;, axis=1) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> level_1 c d</span>
<span class="sd"> a b</span>
<span class="sd"> 1 2 3 4</span>
<span class="sd"> 5 6 7 8</span>
<span class="sd"> 9 10 11 12</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">level</span><span class="p">,</span> <span class="p">(</span><span class="nb">tuple</span><span class="p">,</span> <span class="nb">list</span><span class="p">)):</span> <span class="c1"># huh?</span>
<span class="n">level</span> <span class="o">=</span> <span class="p">[</span><span class="n">level</span><span class="p">]</span>
<span class="n">names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span>
<span class="n">nlevels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span>
<span class="n">int_level</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="k">for</span> <span class="n">n</span> <span class="ow">in</span> <span class="n">level</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">n</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="k">if</span> <span class="n">n</span> <span class="o">&lt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">n</span> <span class="o">=</span> <span class="n">n</span> <span class="o">+</span> <span class="n">nlevels</span>
<span class="k">if</span> <span class="n">n</span> <span class="o">&lt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">IndexError</span><span class="p">(</span>
<span class="s2">&quot;Too many levels: Index has only </span><span class="si">{}</span><span class="s2"> levels, &quot;</span>
<span class="s2">&quot;</span><span class="si">{}</span><span class="s2"> is not a valid level number&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">nlevels</span><span class="p">,</span> <span class="p">(</span><span class="n">n</span> <span class="o">-</span> <span class="n">nlevels</span><span class="p">))</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">n</span> <span class="o">&gt;=</span> <span class="n">nlevels</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">IndexError</span><span class="p">(</span>
<span class="s2">&quot;Too many levels: Index has only </span><span class="si">{}</span><span class="s2"> levels, not </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">nlevels</span><span class="p">,</span> <span class="p">(</span><span class="n">n</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="n">n</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">names</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="s2">&quot;Level </span><span class="si">{}</span><span class="s2"> not found&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">n</span><span class="p">))</span>
<span class="n">n</span> <span class="o">=</span> <span class="n">names</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">n</span><span class="p">)</span>
<span class="n">int_level</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">n</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">level</span><span class="p">)</span> <span class="o">&gt;=</span> <span class="n">nlevels</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Cannot remove </span><span class="si">{}</span><span class="s2"> levels from an index with </span><span class="si">{}</span><span class="s2"> levels: &quot;</span>
<span class="s2">&quot;at least one level must be left.&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">level</span><span class="p">),</span> <span class="n">nlevels</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">index_spark_columns</span><span class="p">,</span> <span class="n">index_names</span><span class="p">,</span> <span class="n">index_fields</span> <span class="o">=</span> <span class="nb">zip</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">item</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">item</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span>
<span class="nb">zip</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">,</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">i</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">int_level</span>
<span class="p">]</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_spark_columns</span><span class="p">),</span>
<span class="n">index_names</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_names</span><span class="p">),</span>
<span class="n">index_fields</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_fields</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">internal</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">columns</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">droplevel</span><span class="p">(</span><span class="n">level</span><span class="p">)</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="k">return</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.drop"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.drop.html#pyspark.pandas.DataFrame.drop">[docs]</a> <span class="k">def</span> <span class="nf">drop</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">labels</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Axis</span><span class="p">]</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">columns</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Drop specified labels from columns.</span>
<span class="sd"> Remove rows and/or columns by specifying label names and corresponding axis,</span>
<span class="sd"> or by specifying directly index and/or column names.</span>
<span class="sd"> Drop rows of a MultiIndex DataFrame is not supported yet.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> labels : single label or list-like</span>
<span class="sd"> Column labels to drop.</span>
<span class="sd"> axis : {0 or &#39;index&#39;, 1 or &#39;columns&#39;}, default 0</span>
<span class="sd"> .. versionchanged:: 3.3</span>
<span class="sd"> Set dropping by index is default.</span>
<span class="sd"> index : single label or list-like</span>
<span class="sd"> Alternative to specifying axis (``labels, axis=0``</span>
<span class="sd"> is equivalent to ``index=columns``).</span>
<span class="sd"> .. versionchanged:: 3.3</span>
<span class="sd"> Added dropping rows by &#39;index&#39;.</span>
<span class="sd"> columns : single label or list-like</span>
<span class="sd"> Alternative to specifying axis (``labels, axis=1``</span>
<span class="sd"> is equivalent to ``columns=labels``).</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> dropped : DataFrame</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.dropna</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(np.arange(12).reshape(3, 4), columns=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;, &#39;D&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C D</span>
<span class="sd"> 0 0 1 2 3</span>
<span class="sd"> 1 4 5 6 7</span>
<span class="sd"> 2 8 9 10 11</span>
<span class="sd"> Drop columns</span>
<span class="sd"> &gt;&gt;&gt; df.drop([&#39;B&#39;, &#39;C&#39;], axis=1)</span>
<span class="sd"> A D</span>
<span class="sd"> 0 0 3</span>
<span class="sd"> 1 4 7</span>
<span class="sd"> 2 8 11</span>
<span class="sd"> &gt;&gt;&gt; df.drop(columns=[&#39;B&#39;, &#39;C&#39;])</span>
<span class="sd"> A D</span>
<span class="sd"> 0 0 3</span>
<span class="sd"> 1 4 7</span>
<span class="sd"> 2 8 11</span>
<span class="sd"> Drop a row by index</span>
<span class="sd"> &gt;&gt;&gt; df.drop([0, 1])</span>
<span class="sd"> A B C D</span>
<span class="sd"> 2 8 9 10 11</span>
<span class="sd"> &gt;&gt;&gt; df.drop(index=[0, 1], columns=&#39;A&#39;)</span>
<span class="sd"> B C D</span>
<span class="sd"> 2 9 10 11</span>
<span class="sd"> Also support dropping columns for MultiIndex</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;x&#39;: [1, 2], &#39;y&#39;: [3, 4], &#39;z&#39;: [5, 6], &#39;w&#39;: [7, 8]},</span>
<span class="sd"> ... columns=[&#39;x&#39;, &#39;y&#39;, &#39;z&#39;, &#39;w&#39;])</span>
<span class="sd"> &gt;&gt;&gt; columns = [(&#39;a&#39;, &#39;x&#39;), (&#39;a&#39;, &#39;y&#39;), (&#39;b&#39;, &#39;z&#39;), (&#39;b&#39;, &#39;w&#39;)]</span>
<span class="sd"> &gt;&gt;&gt; df.columns = pd.MultiIndex.from_tuples(columns)</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> a b</span>
<span class="sd"> x y z w</span>
<span class="sd"> 0 1 3 5 7</span>
<span class="sd"> 1 2 4 6 8</span>
<span class="sd"> &gt;&gt;&gt; df.drop(labels=&#39;a&#39;, axis=1) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> b</span>
<span class="sd"> z w</span>
<span class="sd"> 0 5 7</span>
<span class="sd"> 1 6 8</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> Currently, dropping rows of a MultiIndex DataFrame is not supported yet.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">labels</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Cannot specify both &#39;labels&#39; and &#39;index&#39;/&#39;columns&#39;&quot;</span><span class="p">)</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">index</span><span class="o">=</span><span class="n">index</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">labels</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">index</span><span class="o">=</span><span class="n">labels</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">columns</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Need to specify at least one of &#39;labels&#39; or &#39;columns&#39; or &#39;index&#39;&quot;</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">index</span><span class="p">)</span> <span class="ow">or</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">index</span><span class="p">):</span>
<span class="n">index</span> <span class="o">=</span> <span class="p">[</span><span class="n">index</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">index</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">if</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_level</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">index</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="n">ps</span><span class="o">.</span><span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.isin_limit&quot;</span><span class="p">):</span>
<span class="n">self_index_type</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span>
<span class="n">cond</span> <span class="o">=</span> <span class="o">~</span><span class="n">internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">isin</span><span class="p">(</span>
<span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">self_index_type</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">index</span><span class="p">]</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">cond</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">index_sdf_col</span> <span class="o">=</span> <span class="s2">&quot;__index&quot;</span>
<span class="n">index_sdf</span> <span class="o">=</span> <span class="n">default_session</span><span class="p">()</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span>
<span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">({</span><span class="n">index_sdf_col</span><span class="p">:</span> <span class="n">index</span><span class="p">})</span>
<span class="p">)</span>
<span class="n">joined_sdf</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">join</span><span class="p">(</span>
<span class="n">other</span><span class="o">=</span><span class="n">F</span><span class="o">.</span><span class="n">broadcast</span><span class="p">(</span><span class="n">index_sdf</span><span class="p">),</span>
<span class="n">on</span><span class="o">=</span><span class="p">(</span>
<span class="n">internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="o">==</span> <span class="n">scol_for</span><span class="p">(</span><span class="n">index_sdf</span><span class="p">,</span> <span class="n">index_sdf_col</span><span class="p">)</span>
<span class="p">),</span>
<span class="n">how</span><span class="o">=</span><span class="s2">&quot;anti&quot;</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">joined_sdf</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span>
<span class="s2">&quot;Drop rows of MultiIndex DataFrame is not supported yet&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">columns</span><span class="p">):</span>
<span class="n">columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">columns</span><span class="p">]</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">columns</span><span class="p">):</span>
<span class="n">columns</span> <span class="o">=</span> <span class="p">[(</span><span class="n">columns</span><span class="p">,)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">col</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">col</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">col</span><span class="p">,)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">columns</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">drop_column_labels</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span>
<span class="n">label</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">columns</span>
<span class="k">if</span> <span class="n">label</span><span class="p">[:</span> <span class="nb">len</span><span class="p">(</span><span class="n">col</span><span class="p">)]</span> <span class="o">==</span> <span class="n">col</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">drop_column_labels</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span>
<span class="n">keep_columns_and_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="p">(</span><span class="n">column</span><span class="p">,</span> <span class="n">label</span><span class="p">)</span>
<span class="k">for</span> <span class="n">column</span><span class="p">,</span> <span class="n">label</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">drop_column_labels</span>
<span class="p">]</span>
<span class="n">cols</span><span class="p">,</span> <span class="n">labels</span> <span class="o">=</span> <span class="p">(</span>
<span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">keep_columns_and_labels</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">keep_columns_and_labels</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span>
<span class="k">else</span> <span class="p">([],</span> <span class="p">[])</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span>
<span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">labels</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">internal</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_prepare_sort_by_scols</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">by</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]])</span> <span class="o">-&gt;</span> <span class="n">List</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">]:</span>
<span class="k">if</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">by</span><span class="p">):</span>
<span class="n">by</span> <span class="o">=</span> <span class="p">[</span><span class="n">by</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">assert</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">by</span><span class="p">),</span> <span class="nb">type</span><span class="p">(</span><span class="n">by</span><span class="p">)</span>
<span class="n">new_by</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">colname</span> <span class="ow">in</span> <span class="n">by</span><span class="p">:</span>
<span class="n">ser</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="n">colname</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">ser</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;The column </span><span class="si">%s</span><span class="s2"> is not unique. For a multi-index, the label must be a tuple &quot;</span>
<span class="s2">&quot;with elements corresponding to each level.&quot;</span> <span class="o">%</span> <span class="n">name_like_string</span><span class="p">(</span><span class="n">colname</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">new_by</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">ser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="p">)</span>
<span class="k">return</span> <span class="n">new_by</span>
<span class="k">def</span> <span class="nf">_sort</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">by</span><span class="p">:</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">],</span>
<span class="n">ascending</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">bool</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">bool</span><span class="p">]],</span>
<span class="n">na_position</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span>
<span class="n">keep</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;first&quot;</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ascending</span><span class="p">,</span> <span class="nb">bool</span><span class="p">):</span>
<span class="n">ascending</span> <span class="o">=</span> <span class="p">[</span><span class="n">ascending</span><span class="p">]</span> <span class="o">*</span> <span class="nb">len</span><span class="p">(</span><span class="n">by</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">ascending</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">len</span><span class="p">(</span><span class="n">by</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Length of ascending (</span><span class="si">{}</span><span class="s2">) != length of by (</span><span class="si">{}</span><span class="s2">)&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">ascending</span><span class="p">),</span> <span class="nb">len</span><span class="p">(</span><span class="n">by</span><span class="p">))</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">na_position</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">(</span><span class="s2">&quot;first&quot;</span><span class="p">,</span> <span class="s2">&quot;last&quot;</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;invalid na_position: &#39;</span><span class="si">{}</span><span class="s2">&#39;&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">na_position</span><span class="p">))</span>
<span class="n">Column</span> <span class="o">=</span> <span class="n">get_column_class</span><span class="p">()</span>
<span class="c1"># Mapper: Get a spark colum</span>
<span class="c1"># n function for (ascending, na_position) combination</span>
<span class="n">mapper</span> <span class="o">=</span> <span class="p">{</span>
<span class="p">(</span><span class="kc">True</span><span class="p">,</span> <span class="s2">&quot;first&quot;</span><span class="p">):</span> <span class="n">Column</span><span class="o">.</span><span class="n">asc_nulls_first</span><span class="p">,</span>
<span class="p">(</span><span class="kc">True</span><span class="p">,</span> <span class="s2">&quot;last&quot;</span><span class="p">):</span> <span class="n">Column</span><span class="o">.</span><span class="n">asc_nulls_last</span><span class="p">,</span>
<span class="p">(</span><span class="kc">False</span><span class="p">,</span> <span class="s2">&quot;first&quot;</span><span class="p">):</span> <span class="n">Column</span><span class="o">.</span><span class="n">desc_nulls_first</span><span class="p">,</span>
<span class="p">(</span><span class="kc">False</span><span class="p">,</span> <span class="s2">&quot;last&quot;</span><span class="p">):</span> <span class="n">Column</span><span class="o">.</span><span class="n">desc_nulls_last</span><span class="p">,</span>
<span class="p">}</span>
<span class="n">by</span> <span class="o">=</span> <span class="p">[</span><span class="n">mapper</span><span class="p">[(</span><span class="n">asc</span><span class="p">,</span> <span class="n">na_position</span><span class="p">)](</span><span class="n">scol</span><span class="p">)</span> <span class="k">for</span> <span class="n">scol</span><span class="p">,</span> <span class="n">asc</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">by</span><span class="p">,</span> <span class="n">ascending</span><span class="p">)]</span>
<span class="n">natural_order_scol</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">NATURAL_ORDER_COLUMN_NAME</span><span class="p">)</span>
<span class="k">if</span> <span class="n">keep</span> <span class="o">==</span> <span class="s2">&quot;last&quot;</span><span class="p">:</span>
<span class="n">natural_order_scol</span> <span class="o">=</span> <span class="n">Column</span><span class="o">.</span><span class="n">desc</span><span class="p">(</span><span class="n">natural_order_scol</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">keep</span> <span class="o">==</span> <span class="s2">&quot;all&quot;</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;`keep`=all is not implemented yet.&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">keep</span> <span class="o">!=</span> <span class="s2">&quot;first&quot;</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;keep must be either &quot;first&quot;, &quot;last&quot; or &quot;all&quot;.&#39;</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">sort</span><span class="p">(</span><span class="o">*</span><span class="n">by</span><span class="p">,</span> <span class="n">natural_order_scol</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">sdf</span><span class="p">))</span>
<div class="viewcode-block" id="DataFrame.sort_values"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.sort_values.html#pyspark.pandas.DataFrame.sort_values">[docs]</a> <span class="k">def</span> <span class="nf">sort_values</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">by</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]],</span>
<span class="n">ascending</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">bool</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">bool</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">na_position</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;last&quot;</span><span class="p">,</span>
<span class="n">ignore_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Sort by the values along either axis.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> by : str or list of str</span>
<span class="sd"> ascending : bool or list of bool, default True</span>
<span class="sd"> Sort ascending vs. descending. Specify list for multiple sort</span>
<span class="sd"> orders. If this is a list of bools, must match the length of</span>
<span class="sd"> the by.</span>
<span class="sd"> inplace : bool, default False</span>
<span class="sd"> if True, perform operation in-place</span>
<span class="sd"> na_position : {&#39;first&#39;, &#39;last&#39;}, default &#39;last&#39;</span>
<span class="sd"> `first` puts NaNs at the beginning, `last` puts NaNs at the end</span>
<span class="sd"> ignore_index : bool, default False</span>
<span class="sd"> If True, the resulting axis will be labeled 0, 1, …, n - 1.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> sorted_obj : DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({</span>
<span class="sd"> ... &#39;col1&#39;: [&#39;A&#39;, &#39;B&#39;, None, &#39;D&#39;, &#39;C&#39;],</span>
<span class="sd"> ... &#39;col2&#39;: [2, 9, 8, 7, 4],</span>
<span class="sd"> ... &#39;col3&#39;: [0, 9, 4, 2, 3],</span>
<span class="sd"> ... },</span>
<span class="sd"> ... columns=[&#39;col1&#39;, &#39;col2&#39;, &#39;col3&#39;],</span>
<span class="sd"> ... index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> col1 col2 col3</span>
<span class="sd"> a A 2 0</span>
<span class="sd"> b B 9 9</span>
<span class="sd"> c None 8 4</span>
<span class="sd"> d D 7 2</span>
<span class="sd"> e C 4 3</span>
<span class="sd"> Sort by col1</span>
<span class="sd"> &gt;&gt;&gt; df.sort_values(by=[&#39;col1&#39;])</span>
<span class="sd"> col1 col2 col3</span>
<span class="sd"> a A 2 0</span>
<span class="sd"> b B 9 9</span>
<span class="sd"> e C 4 3</span>
<span class="sd"> d D 7 2</span>
<span class="sd"> c None 8 4</span>
<span class="sd"> Ignore index for the resulting axis</span>
<span class="sd"> &gt;&gt;&gt; df.sort_values(by=[&#39;col1&#39;], ignore_index=True)</span>
<span class="sd"> col1 col2 col3</span>
<span class="sd"> 0 A 2 0</span>
<span class="sd"> 1 B 9 9</span>
<span class="sd"> 2 C 4 3</span>
<span class="sd"> 3 D 7 2</span>
<span class="sd"> 4 None 8 4</span>
<span class="sd"> Sort Descending</span>
<span class="sd"> &gt;&gt;&gt; df.sort_values(by=&#39;col1&#39;, ascending=False)</span>
<span class="sd"> col1 col2 col3</span>
<span class="sd"> d D 7 2</span>
<span class="sd"> e C 4 3</span>
<span class="sd"> b B 9 9</span>
<span class="sd"> a A 2 0</span>
<span class="sd"> c None 8 4</span>
<span class="sd"> Sort by multiple columns</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({</span>
<span class="sd"> ... &#39;col1&#39;: [&#39;A&#39;, &#39;A&#39;, &#39;B&#39;, None, &#39;D&#39;, &#39;C&#39;],</span>
<span class="sd"> ... &#39;col2&#39;: [2, 1, 9, 8, 7, 4],</span>
<span class="sd"> ... &#39;col3&#39;: [0, 1, 9, 4, 2, 3],</span>
<span class="sd"> ... },</span>
<span class="sd"> ... columns=[&#39;col1&#39;, &#39;col2&#39;, &#39;col3&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.sort_values(by=[&#39;col1&#39;, &#39;col2&#39;])</span>
<span class="sd"> col1 col2 col3</span>
<span class="sd"> 1 A 1 1</span>
<span class="sd"> 0 A 2 0</span>
<span class="sd"> 2 B 9 9</span>
<span class="sd"> 5 C 4 3</span>
<span class="sd"> 4 D 7 2</span>
<span class="sd"> 3 None 8 4</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="n">new_by</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_prepare_sort_by_scols</span><span class="p">(</span><span class="n">by</span><span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sort</span><span class="p">(</span><span class="n">by</span><span class="o">=</span><span class="n">new_by</span><span class="p">,</span> <span class="n">ascending</span><span class="o">=</span><span class="n">ascending</span><span class="p">,</span> <span class="n">na_position</span><span class="o">=</span><span class="n">na_position</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="k">if</span> <span class="n">ignore_index</span><span class="p">:</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">inplace</span><span class="o">=</span><span class="n">inplace</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> <span class="k">if</span> <span class="n">ignore_index</span> <span class="k">else</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.sort_index"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.sort_index.html#pyspark.pandas.DataFrame.sort_index">[docs]</a> <span class="k">def</span> <span class="nf">sort_index</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="n">level</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">int</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">ascending</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">kind</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">na_position</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;last&quot;</span><span class="p">,</span>
<span class="n">ignore_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Sort object by labels (along an axis)</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> axis : index, columns to direct sorting. Currently, only axis = 0 is supported.</span>
<span class="sd"> level : int or level name or list of ints or list of level names</span>
<span class="sd"> if not None, sort on values in specified index level(s)</span>
<span class="sd"> ascending : boolean, default True</span>
<span class="sd"> Sort ascending vs. descending</span>
<span class="sd"> inplace : bool, default False</span>
<span class="sd"> if True, perform operation in-place</span>
<span class="sd"> kind : str, default None</span>
<span class="sd"> pandas-on-Spark does not allow specifying the sorting algorithm now,</span>
<span class="sd"> default None</span>
<span class="sd"> na_position : {‘first’, ‘last’}, default ‘last’</span>
<span class="sd"> first puts NaNs at the beginning, last puts NaNs at the end. Not implemented for</span>
<span class="sd"> MultiIndex.</span>
<span class="sd"> ignore_index : bool, default False</span>
<span class="sd"> If True, the resulting axis will be labeled 0, 1, …, n - 1.</span>
<span class="sd"> .. versionadded:: 3.4.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> sorted_obj : DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [2, 1, np.nan]}, index=[&#39;b&#39;, &#39;a&#39;, np.nan])</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index() # doctest: +SKIP</span>
<span class="sd"> A</span>
<span class="sd"> a 1.0</span>
<span class="sd"> b 2.0</span>
<span class="sd"> None NaN</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index(ascending=False) # doctest: +SKIP</span>
<span class="sd"> A</span>
<span class="sd"> b 2.0</span>
<span class="sd"> a 1.0</span>
<span class="sd"> None NaN</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index(na_position=&#39;first&#39;) # doctest: +SKIP</span>
<span class="sd"> A</span>
<span class="sd"> None NaN</span>
<span class="sd"> a 1.0</span>
<span class="sd"> b 2.0</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index(ignore_index=True)</span>
<span class="sd"> A</span>
<span class="sd"> 0 1.0</span>
<span class="sd"> 1 2.0</span>
<span class="sd"> 2 NaN</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index(inplace=True)</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +SKIP</span>
<span class="sd"> A</span>
<span class="sd"> a 1.0</span>
<span class="sd"> b 2.0</span>
<span class="sd"> None NaN</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: range(4), &#39;B&#39;: range(4)[::-1]},</span>
<span class="sd"> ... index=[[&#39;b&#39;, &#39;b&#39;, &#39;a&#39;, &#39;a&#39;], [1, 0, 1, 0]],</span>
<span class="sd"> ... columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> a 0 3 0</span>
<span class="sd"> 1 2 1</span>
<span class="sd"> b 0 1 2</span>
<span class="sd"> 1 0 3</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index(level=1)</span>
<span class="sd"> A B</span>
<span class="sd"> b 0 1 2</span>
<span class="sd"> a 0 3 0</span>
<span class="sd"> b 1 0 3</span>
<span class="sd"> a 1 2 1</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index(level=[1, 0])</span>
<span class="sd"> A B</span>
<span class="sd"> a 0 3 0</span>
<span class="sd"> b 0 1 2</span>
<span class="sd"> a 1 2 1</span>
<span class="sd"> b 1 0 3</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index(ignore_index=True)</span>
<span class="sd"> A B</span>
<span class="sd"> 0 3 0</span>
<span class="sd"> 1 2 1</span>
<span class="sd"> 2 1 2</span>
<span class="sd"> 3 0 3</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;No other axis than 0 are supported now&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">kind</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;Specifying the sorting algorithm is not supported now.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="p">(</span><span class="n">is_list_like</span><span class="p">(</span><span class="n">level</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">level</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">):</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="n">by</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="k">elif</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">level</span><span class="p">):</span>
<span class="n">by</span> <span class="o">=</span> <span class="p">[</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="n">lvl</span><span class="p">]</span> <span class="k">for</span> <span class="n">lvl</span> <span class="ow">in</span> <span class="n">level</span> <span class="c1"># type: ignore[union-attr]</span>
<span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">by</span> <span class="o">=</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="n">level</span><span class="p">]]</span> <span class="c1"># type: ignore[index]</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sort</span><span class="p">(</span><span class="n">by</span><span class="o">=</span><span class="n">by</span><span class="p">,</span> <span class="n">ascending</span><span class="o">=</span><span class="n">ascending</span><span class="p">,</span> <span class="n">na_position</span><span class="o">=</span><span class="n">na_position</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="k">if</span> <span class="n">ignore_index</span><span class="p">:</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">inplace</span><span class="o">=</span><span class="n">inplace</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> <span class="k">if</span> <span class="n">ignore_index</span> <span class="k">else</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.swaplevel"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.swaplevel.html#pyspark.pandas.DataFrame.swaplevel">[docs]</a> <span class="k">def</span> <span class="nf">swaplevel</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">i</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">]</span> <span class="o">=</span> <span class="o">-</span><span class="mi">2</span><span class="p">,</span> <span class="n">j</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">]</span> <span class="o">=</span> <span class="o">-</span><span class="mi">1</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Swap levels i and j in a MultiIndex on a particular axis.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> i, j : int or str</span>
<span class="sd"> Levels of the indices to be swapped. Can pass level name as string.</span>
<span class="sd"> axis : {0 or &#39;index&#39;, 1 or &#39;columns&#39;}, default 0</span>
<span class="sd"> The axis to swap levels on. 0 or &#39;index&#39; for row-wise, 1 or</span>
<span class="sd"> &#39;columns&#39; for column-wise.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame with levels swapped in MultiIndex.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; midx = pd.MultiIndex.from_arrays(</span>
<span class="sd"> ... [[&#39;red&#39;, &#39;blue&#39;], [1, 2], [&#39;s&#39;, &#39;m&#39;]], names = [&#39;color&#39;, &#39;number&#39;, &#39;size&#39;])</span>
<span class="sd"> &gt;&gt;&gt; midx # doctest: +SKIP</span>
<span class="sd"> MultiIndex([( &#39;red&#39;, 1, &#39;s&#39;),</span>
<span class="sd"> (&#39;blue&#39;, 2, &#39;m&#39;)],</span>
<span class="sd"> names=[&#39;color&#39;, &#39;number&#39;, &#39;size&#39;])</span>
<span class="sd"> Swap levels in a MultiIndex on index.</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;x&#39;: [5, 6], &#39;y&#39;:[5, 6]}, index=midx)</span>
<span class="sd"> &gt;&gt;&gt; psdf # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> x y</span>
<span class="sd"> color number size</span>
<span class="sd"> red 1 s 5 5</span>
<span class="sd"> blue 2 m 6 6</span>
<span class="sd"> &gt;&gt;&gt; psdf.swaplevel() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> x y</span>
<span class="sd"> color size number</span>
<span class="sd"> red s 1 5 5</span>
<span class="sd"> blue m 2 6 6</span>
<span class="sd"> &gt;&gt;&gt; psdf.swaplevel(0, 1) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> x y</span>
<span class="sd"> number color size</span>
<span class="sd"> 1 red s 5 5</span>
<span class="sd"> 2 blue m 6 6</span>
<span class="sd"> &gt;&gt;&gt; psdf.swaplevel(&#39;number&#39;, &#39;size&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> x y</span>
<span class="sd"> color size number</span>
<span class="sd"> red s 1 5 5</span>
<span class="sd"> blue m 2 6 6</span>
<span class="sd"> Swap levels in a MultiIndex on columns.</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;x&#39;: [5, 6], &#39;y&#39;:[5, 6]})</span>
<span class="sd"> &gt;&gt;&gt; psdf.columns = midx</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> color red blue</span>
<span class="sd"> number 1 2</span>
<span class="sd"> size s m</span>
<span class="sd"> 0 5 5</span>
<span class="sd"> 1 6 6</span>
<span class="sd"> &gt;&gt;&gt; psdf.swaplevel(axis=1)</span>
<span class="sd"> color red blue</span>
<span class="sd"> size s m</span>
<span class="sd"> number 1 2</span>
<span class="sd"> 0 5 5</span>
<span class="sd"> 1 6 6</span>
<span class="sd"> &gt;&gt;&gt; psdf.swaplevel(axis=1)</span>
<span class="sd"> color red blue</span>
<span class="sd"> size s m</span>
<span class="sd"> number 1 2</span>
<span class="sd"> 0 5 5</span>
<span class="sd"> 1 6 6</span>
<span class="sd"> &gt;&gt;&gt; psdf.swaplevel(0, 1, axis=1)</span>
<span class="sd"> number 1 2</span>
<span class="sd"> color red blue</span>
<span class="sd"> size s m</span>
<span class="sd"> 0 5 5</span>
<span class="sd"> 1 6 6</span>
<span class="sd"> &gt;&gt;&gt; psdf.swaplevel(&#39;number&#39;, &#39;color&#39;, axis=1)</span>
<span class="sd"> number 1 2</span>
<span class="sd"> color red blue</span>
<span class="sd"> size s m</span>
<span class="sd"> 0 5 5</span>
<span class="sd"> 1 6 6</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_swaplevel_index</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="n">j</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">assert</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">1</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_swaplevel_columns</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="n">j</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.swapaxes"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.swapaxes.html#pyspark.pandas.DataFrame.swapaxes">[docs]</a> <span class="k">def</span> <span class="nf">swapaxes</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">i</span><span class="p">:</span> <span class="n">Axis</span><span class="p">,</span> <span class="n">j</span><span class="p">:</span> <span class="n">Axis</span><span class="p">,</span> <span class="n">copy</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Interchange axes and swap values axes appropriately.</span>
<span class="sd"> .. note:: This method is based on an expensive operation due to the nature</span>
<span class="sd"> of big data. Internally it needs to generate each row for each value, and</span>
<span class="sd"> then group twice - it is a huge operation. To prevent misuse, this method</span>
<span class="sd"> has the &#39;compute.max_rows&#39; default limit of input length and raises a ValueError.</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.pandas.config import option_context</span>
<span class="sd"> &gt;&gt;&gt; with option_context(&#39;compute.max_rows&#39;, 1000): # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> ... ps.DataFrame({&#39;a&#39;: range(1001)}).swapaxes(i=0, j=1)</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: Current DataFrame&#39;s length exceeds the given limit of 1000 rows.</span>
<span class="sd"> Please set &#39;compute.max_rows&#39; by using &#39;pyspark.pandas.config.set_option&#39;</span>
<span class="sd"> to retrieve more than 1000 rows. Note that, before changing the</span>
<span class="sd"> &#39;compute.max_rows&#39;, this operation is considerably expensive.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> i: {0 or &#39;index&#39;, 1 or &#39;columns&#39;}. The axis to swap.</span>
<span class="sd"> j: {0 or &#39;index&#39;, 1 or &#39;columns&#39;}. The axis to swap.</span>
<span class="sd"> copy : bool, default True.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame(</span>
<span class="sd"> ... [[1, 2, 3], [4, 5, 6], [7, 8, 9]], index=[&#39;x&#39;, &#39;y&#39;, &#39;z&#39;], columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;]</span>
<span class="sd"> ... )</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> a b c</span>
<span class="sd"> x 1 2 3</span>
<span class="sd"> y 4 5 6</span>
<span class="sd"> z 7 8 9</span>
<span class="sd"> &gt;&gt;&gt; psdf.swapaxes(i=1, j=0)</span>
<span class="sd"> x y z</span>
<span class="sd"> a 1 4 7</span>
<span class="sd"> b 2 5 8</span>
<span class="sd"> c 3 6 9</span>
<span class="sd"> &gt;&gt;&gt; psdf.swapaxes(i=1, j=1)</span>
<span class="sd"> a b c</span>
<span class="sd"> x 1 2 3</span>
<span class="sd"> y 4 5 6</span>
<span class="sd"> z 7 8 9</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="n">copy</span> <span class="ow">is</span> <span class="kc">True</span>
<span class="n">i</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="n">j</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">j</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span> <span class="k">if</span> <span class="n">i</span> <span class="o">==</span> <span class="n">j</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">transpose</span><span class="p">()</span></div>
<span class="k">def</span> <span class="nf">_swaplevel_columns</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">i</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">],</span> <span class="n">j</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="n">InternalFrame</span><span class="p">:</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">)</span>
<span class="k">for</span> <span class="n">index</span> <span class="ow">in</span> <span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="n">j</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">index</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="ow">and</span> <span class="n">index</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">names</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="s2">&quot;Level </span><span class="si">%s</span><span class="s2"> not found&quot;</span> <span class="o">%</span> <span class="n">index</span><span class="p">)</span>
<span class="n">i</span> <span class="o">=</span> <span class="n">i</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">names</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="n">j</span> <span class="o">=</span> <span class="n">j</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">j</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">names</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">j</span><span class="p">)</span>
<span class="k">for</span> <span class="n">index</span> <span class="ow">in</span> <span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="n">j</span><span class="p">):</span>
<span class="k">if</span> <span class="n">index</span> <span class="o">&gt;=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span> <span class="ow">or</span> <span class="n">index</span> <span class="o">&lt;</span> <span class="o">-</span><span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">IndexError</span><span class="p">(</span>
<span class="s2">&quot;Too many levels: Columns have only </span><span class="si">%s</span><span class="s2"> levels, &quot;</span>
<span class="s2">&quot;</span><span class="si">%s</span><span class="s2"> is not a valid level number&quot;</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">,</span> <span class="n">index</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="p">(</span>
<span class="n">column_label_names</span><span class="p">[</span><span class="n">i</span><span class="p">],</span>
<span class="n">column_label_names</span><span class="p">[</span><span class="n">j</span><span class="p">],</span>
<span class="p">)</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">column_label_names</span><span class="p">[</span><span class="n">j</span><span class="p">],</span>
<span class="n">column_label_names</span><span class="p">[</span><span class="n">i</span><span class="p">],</span>
<span class="p">)</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">_column_labels</span>
<span class="n">column_label_list</span> <span class="o">=</span> <span class="p">[</span><span class="nb">list</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">]</span>
<span class="k">for</span> <span class="n">label_list</span> <span class="ow">in</span> <span class="n">column_label_list</span><span class="p">:</span>
<span class="n">label_list</span><span class="p">[</span><span class="n">i</span><span class="p">],</span> <span class="n">label_list</span><span class="p">[</span><span class="n">j</span><span class="p">]</span> <span class="o">=</span> <span class="n">label_list</span><span class="p">[</span><span class="n">j</span><span class="p">],</span> <span class="n">label_list</span><span class="p">[</span><span class="n">i</span><span class="p">]</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span><span class="nb">tuple</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">column_label_list</span><span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">column_label_names</span><span class="p">),</span> <span class="n">column_labels</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">internal</span>
<span class="k">def</span> <span class="nf">_swaplevel_index</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">i</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">],</span> <span class="n">j</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="n">Name</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="n">InternalFrame</span><span class="p">:</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">)</span>
<span class="k">for</span> <span class="n">index</span> <span class="ow">in</span> <span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="n">j</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">index</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="ow">and</span> <span class="n">index</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="s2">&quot;Level </span><span class="si">%s</span><span class="s2"> not found&quot;</span> <span class="o">%</span> <span class="n">index</span><span class="p">)</span>
<span class="n">i</span> <span class="o">=</span> <span class="n">i</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="n">j</span> <span class="o">=</span> <span class="n">j</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">j</span><span class="p">,</span> <span class="nb">int</span><span class="p">)</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">j</span><span class="p">)</span>
<span class="k">for</span> <span class="n">index</span> <span class="ow">in</span> <span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="n">j</span><span class="p">):</span>
<span class="k">if</span> <span class="n">index</span> <span class="o">&gt;=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span> <span class="ow">or</span> <span class="n">index</span> <span class="o">&lt;</span> <span class="o">-</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">IndexError</span><span class="p">(</span>
<span class="s2">&quot;Too many levels: Index has only </span><span class="si">%s</span><span class="s2"> levels, &quot;</span>
<span class="s2">&quot;</span><span class="si">%s</span><span class="s2"> is not a valid level number&quot;</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">,</span> <span class="n">index</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">index_map</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span>
<span class="nb">zip</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">,</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">index_map</span><span class="p">[</span><span class="n">i</span><span class="p">],</span> <span class="n">index_map</span><span class="p">[</span><span class="n">j</span><span class="p">]</span> <span class="o">=</span> <span class="n">index_map</span><span class="p">[</span><span class="n">j</span><span class="p">],</span> <span class="n">index_map</span><span class="p">[</span><span class="n">i</span><span class="p">]</span>
<span class="n">index_spark_columns</span><span class="p">,</span> <span class="n">index_names</span><span class="p">,</span> <span class="n">index_fields</span> <span class="o">=</span> <span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">index_map</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_spark_columns</span><span class="p">),</span>
<span class="n">index_names</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_names</span><span class="p">),</span>
<span class="n">index_fields</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_fields</span><span class="p">),</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">internal</span>
<div class="viewcode-block" id="DataFrame.nlargest"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.nlargest.html#pyspark.pandas.DataFrame.nlargest">[docs]</a> <span class="k">def</span> <span class="nf">nlargest</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">:</span> <span class="nb">int</span><span class="p">,</span> <span class="n">columns</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]],</span> <span class="n">keep</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;first&quot;</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the first `n` rows ordered by `columns` in descending order.</span>
<span class="sd"> Return the first `n` rows with the largest values in `columns`, in</span>
<span class="sd"> descending order. The columns that are not specified are returned as</span>
<span class="sd"> well, but not used for ordering.</span>
<span class="sd"> This method is equivalent to</span>
<span class="sd"> ``df.sort_values(columns, ascending=False).head(n)``, but more</span>
<span class="sd"> performant in pandas.</span>
<span class="sd"> In pandas-on-Spark, thanks to Spark&#39;s lazy execution and query optimizer,</span>
<span class="sd"> the two would have same performance.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> n : int</span>
<span class="sd"> Number of rows to return.</span>
<span class="sd"> columns : label or list of labels</span>
<span class="sd"> Column label(s) to order by.</span>
<span class="sd"> keep : {&#39;first&#39;, &#39;last&#39;}, default &#39;first&#39;. &#39;all&#39; is not implemented yet.</span>
<span class="sd"> Determines which duplicates (if any) to keep.</span>
<span class="sd"> - ``first`` : Keep the first occurrence.</span>
<span class="sd"> - ``last`` : Keep the last occurrence.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> The first `n` rows ordered by the given columns in descending</span>
<span class="sd"> order.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.nsmallest : Return the first `n` rows ordered by `columns` in</span>
<span class="sd"> ascending order.</span>
<span class="sd"> DataFrame.sort_values : Sort DataFrame by the values.</span>
<span class="sd"> DataFrame.head : Return the first `n` rows without re-ordering.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This function cannot be used with all column types. For example, when</span>
<span class="sd"> specifying columns with `object` or `category` dtypes, ``TypeError`` is</span>
<span class="sd"> raised.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;X&#39;: [1, 2, 3, 5, 6, 7, np.nan],</span>
<span class="sd"> ... &#39;Y&#39;: [6, 7, 8, 9, 10, 11, 12]})</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> X Y</span>
<span class="sd"> 0 1.0 6</span>
<span class="sd"> 1 2.0 7</span>
<span class="sd"> 2 3.0 8</span>
<span class="sd"> 3 5.0 9</span>
<span class="sd"> 4 6.0 10</span>
<span class="sd"> 5 7.0 11</span>
<span class="sd"> 6 NaN 12</span>
<span class="sd"> In the following example, we will use ``nlargest`` to select the three</span>
<span class="sd"> rows having the largest values in column &quot;X&quot;.</span>
<span class="sd"> &gt;&gt;&gt; df.nlargest(n=3, columns=&#39;X&#39;)</span>
<span class="sd"> X Y</span>
<span class="sd"> 5 7.0 11</span>
<span class="sd"> 4 6.0 10</span>
<span class="sd"> 3 5.0 9</span>
<span class="sd"> To order by the largest values in column &quot;Y&quot; and then &quot;X&quot;, we can</span>
<span class="sd"> specify multiple columns like in the next example.</span>
<span class="sd"> &gt;&gt;&gt; df.nlargest(n=3, columns=[&#39;Y&#39;, &#39;X&#39;])</span>
<span class="sd"> X Y</span>
<span class="sd"> 6 NaN 12</span>
<span class="sd"> 5 7.0 11</span>
<span class="sd"> 4 6.0 10</span>
<span class="sd"> The examples below show how ties are resolved, which is decided by `keep`.</span>
<span class="sd"> &gt;&gt;&gt; tied_df = ps.DataFrame({&#39;X&#39;: [1, 2, 2, 3, 3]}, index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> &gt;&gt;&gt; tied_df</span>
<span class="sd"> X</span>
<span class="sd"> a 1</span>
<span class="sd"> b 2</span>
<span class="sd"> c 2</span>
<span class="sd"> d 3</span>
<span class="sd"> e 3</span>
<span class="sd"> When using keep=&#39;first&#39; (default), ties are resolved in order:</span>
<span class="sd"> &gt;&gt;&gt; tied_df.nlargest(3, &#39;X&#39;)</span>
<span class="sd"> X</span>
<span class="sd"> d 3</span>
<span class="sd"> e 3</span>
<span class="sd"> b 2</span>
<span class="sd"> &gt;&gt;&gt; tied_df.nlargest(3, &#39;X&#39;, keep=&#39;first&#39;)</span>
<span class="sd"> X</span>
<span class="sd"> d 3</span>
<span class="sd"> e 3</span>
<span class="sd"> b 2</span>
<span class="sd"> When using keep=&#39;last&#39;, ties are resolved in reverse order:</span>
<span class="sd"> &gt;&gt;&gt; tied_df.nlargest(3, &#39;X&#39;, keep=&#39;last&#39;)</span>
<span class="sd"> X</span>
<span class="sd"> e 3</span>
<span class="sd"> d 3</span>
<span class="sd"> c 2</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">by_scols</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_prepare_sort_by_scols</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sort</span><span class="p">(</span><span class="n">by</span><span class="o">=</span><span class="n">by_scols</span><span class="p">,</span> <span class="n">ascending</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">na_position</span><span class="o">=</span><span class="s2">&quot;last&quot;</span><span class="p">,</span> <span class="n">keep</span><span class="o">=</span><span class="n">keep</span><span class="p">)</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">n</span><span class="o">=</span><span class="n">n</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.nsmallest"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.nsmallest.html#pyspark.pandas.DataFrame.nsmallest">[docs]</a> <span class="k">def</span> <span class="nf">nsmallest</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">:</span> <span class="nb">int</span><span class="p">,</span> <span class="n">columns</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]],</span> <span class="n">keep</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;first&quot;</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the first `n` rows ordered by `columns` in ascending order.</span>
<span class="sd"> Return the first `n` rows with the smallest values in `columns`, in</span>
<span class="sd"> ascending order. The columns that are not specified are returned as</span>
<span class="sd"> well, but not used for ordering.</span>
<span class="sd"> This method is equivalent to ``df.sort_values(columns, ascending=True).head(n)``,</span>
<span class="sd"> but more performant. In pandas-on-Spark, thanks to Spark&#39;s lazy execution and query</span>
<span class="sd"> optimizer, the two would have same performance.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> n : int</span>
<span class="sd"> Number of items to retrieve.</span>
<span class="sd"> columns : list or str</span>
<span class="sd"> Column name or names to order by.</span>
<span class="sd"> keep : {&#39;first&#39;, &#39;last&#39;}, default &#39;first&#39;. &#39;all&#39; is not implemented yet.</span>
<span class="sd"> Determines which duplicates (if any) to keep.</span>
<span class="sd"> - ``first`` : Keep the first occurrence.</span>
<span class="sd"> - ``last`` : Keep the last occurrence.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.nlargest : Return the first `n` rows ordered by `columns` in</span>
<span class="sd"> descending order.</span>
<span class="sd"> DataFrame.sort_values : Sort DataFrame by the values.</span>
<span class="sd"> DataFrame.head : Return the first `n` rows without re-ordering.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;X&#39;: [1, 2, 3, 5, 6, 7, np.nan],</span>
<span class="sd"> ... &#39;Y&#39;: [6, 7, 8, 9, 10, 11, 12]})</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> X Y</span>
<span class="sd"> 0 1.0 6</span>
<span class="sd"> 1 2.0 7</span>
<span class="sd"> 2 3.0 8</span>
<span class="sd"> 3 5.0 9</span>
<span class="sd"> 4 6.0 10</span>
<span class="sd"> 5 7.0 11</span>
<span class="sd"> 6 NaN 12</span>
<span class="sd"> In the following example, we will use ``nsmallest`` to select the</span>
<span class="sd"> three rows having the smallest values in column &quot;X&quot;.</span>
<span class="sd"> &gt;&gt;&gt; df.nsmallest(n=3, columns=&#39;X&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> X Y</span>
<span class="sd"> 0 1.0 6</span>
<span class="sd"> 1 2.0 7</span>
<span class="sd"> 2 3.0 8</span>
<span class="sd"> To order by the smallest values in column &quot;Y&quot; and then &quot;X&quot;, we can</span>
<span class="sd"> specify multiple columns like in the next example.</span>
<span class="sd"> &gt;&gt;&gt; df.nsmallest(n=3, columns=[&#39;Y&#39;, &#39;X&#39;]) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> X Y</span>
<span class="sd"> 0 1.0 6</span>
<span class="sd"> 1 2.0 7</span>
<span class="sd"> 2 3.0 8</span>
<span class="sd"> The examples below show how ties are resolved, which is decided by `keep`.</span>
<span class="sd"> &gt;&gt;&gt; tied_df = ps.DataFrame({&#39;X&#39;: [1, 1, 2, 2, 3]}, index=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> &gt;&gt;&gt; tied_df</span>
<span class="sd"> X</span>
<span class="sd"> a 1</span>
<span class="sd"> b 1</span>
<span class="sd"> c 2</span>
<span class="sd"> d 2</span>
<span class="sd"> e 3</span>
<span class="sd"> When using keep=&#39;first&#39; (default), ties are resolved in order:</span>
<span class="sd"> &gt;&gt;&gt; tied_df.nsmallest(3, &#39;X&#39;)</span>
<span class="sd"> X</span>
<span class="sd"> a 1</span>
<span class="sd"> b 1</span>
<span class="sd"> c 2</span>
<span class="sd"> &gt;&gt;&gt; tied_df.nsmallest(3, &#39;X&#39;, keep=&#39;first&#39;)</span>
<span class="sd"> X</span>
<span class="sd"> a 1</span>
<span class="sd"> b 1</span>
<span class="sd"> c 2</span>
<span class="sd"> When using keep=&#39;last&#39;, ties are resolved in reverse order:</span>
<span class="sd"> &gt;&gt;&gt; tied_df.nsmallest(3, &#39;X&#39;, keep=&#39;last&#39;)</span>
<span class="sd"> X</span>
<span class="sd"> b 1</span>
<span class="sd"> a 1</span>
<span class="sd"> d 2</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">by_scols</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_prepare_sort_by_scols</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sort</span><span class="p">(</span><span class="n">by</span><span class="o">=</span><span class="n">by_scols</span><span class="p">,</span> <span class="n">ascending</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">na_position</span><span class="o">=</span><span class="s2">&quot;last&quot;</span><span class="p">,</span> <span class="n">keep</span><span class="o">=</span><span class="n">keep</span><span class="p">)</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">n</span><span class="o">=</span><span class="n">n</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.isin"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.isin.html#pyspark.pandas.DataFrame.isin">[docs]</a> <span class="k">def</span> <span class="nf">isin</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">values</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">List</span><span class="p">,</span> <span class="n">Dict</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Whether each element in the DataFrame is contained in values.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> values : iterable or dict</span>
<span class="sd"> The sequence of values to test. If values are a dict,</span>
<span class="sd"> the keys must be the column names, which must match.</span>
<span class="sd"> Series and DataFrame are not supported.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame of booleans showing whether each element in the DataFrame</span>
<span class="sd"> is contained in values.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;num_legs&#39;: [2, 4], &#39;num_wings&#39;: [2, 0]},</span>
<span class="sd"> ... index=[&#39;falcon&#39;, &#39;dog&#39;],</span>
<span class="sd"> ... columns=[&#39;num_legs&#39;, &#39;num_wings&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> num_legs num_wings</span>
<span class="sd"> falcon 2 2</span>
<span class="sd"> dog 4 0</span>
<span class="sd"> When ``values`` is a list check whether every value in the DataFrame</span>
<span class="sd"> is present in the list (which animals have 0 or 2 legs or wings)</span>
<span class="sd"> &gt;&gt;&gt; df.isin([0, 2])</span>
<span class="sd"> num_legs num_wings</span>
<span class="sd"> falcon True True</span>
<span class="sd"> dog False True</span>
<span class="sd"> When ``values`` is a dict, we can pass values to check for each</span>
<span class="sd"> column separately:</span>
<span class="sd"> &gt;&gt;&gt; df.isin({&#39;num_wings&#39;: [0, 3]})</span>
<span class="sd"> num_legs num_wings</span>
<span class="sd"> falcon False False</span>
<span class="sd"> dog False True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="p">(</span><span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;DataFrame and Series are not supported&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="nb">dict</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">set</span><span class="p">(</span><span class="n">values</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span><span class="o">.</span><span class="n">issubset</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">AttributeError</span><span class="p">(</span>
<span class="s2">&quot;&#39;DataFrame&#39; object has no attribute </span><span class="si">%s</span><span class="s2">&quot;</span>
<span class="o">%</span> <span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">values</span><span class="o">.</span><span class="n">keys</span><span class="p">())</span><span class="o">.</span><span class="n">difference</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">))</span>
<span class="p">)</span>
<span class="n">data_spark_columns</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">col</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">):</span>
<span class="k">if</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">values</span><span class="p">:</span>
<span class="n">item</span> <span class="o">=</span> <span class="n">values</span><span class="p">[</span><span class="n">col</span><span class="p">]</span>
<span class="n">item</span> <span class="o">=</span> <span class="n">item</span><span class="o">.</span><span class="n">tolist</span><span class="p">()</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">item</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">)</span> <span class="k">else</span> <span class="nb">list</span><span class="p">(</span><span class="n">item</span><span class="p">)</span>
<span class="n">scol</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">[</span><span class="n">i</span><span class="p">])</span><span class="o">.</span><span class="n">isin</span><span class="p">(</span>
<span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">item</span><span class="p">]</span>
<span class="p">)</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">coalesce</span><span class="p">(</span><span class="n">scol</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">)</span>
<span class="n">data_spark_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span><span class="p">[</span><span class="n">i</span><span class="p">]))</span>
<span class="k">elif</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">values</span><span class="p">):</span>
<span class="n">values</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">cast</span><span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">,</span> <span class="n">values</span><span class="p">)</span><span class="o">.</span><span class="n">tolist</span><span class="p">()</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">)</span>
<span class="k">else</span> <span class="nb">list</span><span class="p">(</span><span class="n">values</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">isin</span><span class="p">([</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">values</span><span class="p">])</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">coalesce</span><span class="p">(</span><span class="n">scol</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">))</span>
<span class="n">data_spark_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Values should be iterable, Series, DataFrame or dict.&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span>
<span class="n">data_spark_columns</span><span class="p">,</span>
<span class="n">data_fields</span><span class="o">=</span><span class="p">[</span>
<span class="n">field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">dtype</span><span class="o">=</span><span class="n">np</span><span class="o">.</span><span class="n">dtype</span><span class="p">(</span><span class="s2">&quot;bool&quot;</span><span class="p">),</span> <span class="n">spark_type</span><span class="o">=</span><span class="n">BooleanType</span><span class="p">(),</span> <span class="n">nullable</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span>
<span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span>
<span class="p">],</span>
<span class="p">)</span>
<span class="p">)</span></div>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">shape</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Tuple</span><span class="p">[</span><span class="nb">int</span><span class="p">,</span> <span class="nb">int</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a tuple representing the dimensionality of the DataFrame.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;col1&#39;: [1, 2], &#39;col2&#39;: [3, 4]})</span>
<span class="sd"> &gt;&gt;&gt; df.shape</span>
<span class="sd"> (2, 2)</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;col1&#39;: [1, 2], &#39;col2&#39;: [3, 4],</span>
<span class="sd"> ... &#39;col3&#39;: [5, 6]})</span>
<span class="sd"> &gt;&gt;&gt; df.shape</span>
<span class="sd"> (2, 3)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="p">),</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.merge"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.merge.html#pyspark.pandas.DataFrame.merge">[docs]</a> <span class="k">def</span> <span class="nf">merge</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">right</span><span class="p">:</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">,</span>
<span class="n">how</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;inner&quot;</span><span class="p">,</span>
<span class="n">on</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">left_on</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">right_on</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">left_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">right_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">suffixes</span><span class="p">:</span> <span class="n">Tuple</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="p">(</span><span class="s2">&quot;_x&quot;</span><span class="p">,</span> <span class="s2">&quot;_y&quot;</span><span class="p">),</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Merge DataFrame objects with a database-style join.</span>
<span class="sd"> The index of the resulting DataFrame will be one of the following:</span>
<span class="sd"> - 0...n if no index is used for merging</span>
<span class="sd"> - Index of the left DataFrame if merged only on the index of the right DataFrame</span>
<span class="sd"> - Index of the right DataFrame if merged only on the index of the left DataFrame</span>
<span class="sd"> - All involved indices if merged using the indices of both DataFrames</span>
<span class="sd"> e.g. if `left` with indices (a, x) and `right` with indices (b, x), the result will</span>
<span class="sd"> be an index (x, a, b)</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> right: Object to merge with.</span>
<span class="sd"> how: Type of merge to be performed.</span>
<span class="sd"> {&#39;left&#39;, &#39;right&#39;, &#39;outer&#39;, &#39;inner&#39;}, default &#39;inner&#39;</span>
<span class="sd"> left: use only keys from left frame, like a SQL left outer join; not preserve</span>
<span class="sd"> key order unlike pandas.</span>
<span class="sd"> right: use only keys from right frame, like a SQL right outer join; not preserve</span>
<span class="sd"> key order unlike pandas.</span>
<span class="sd"> outer: use union of keys from both frames, like a SQL full outer join; sort keys</span>
<span class="sd"> lexicographically.</span>
<span class="sd"> inner: use intersection of keys from both frames, like a SQL inner join;</span>
<span class="sd"> not preserve the order of the left keys unlike pandas.</span>
<span class="sd"> on: Column or index level names to join on. These must be found in both DataFrames. If on</span>
<span class="sd"> is None and not merging on indexes then this defaults to the intersection of the</span>
<span class="sd"> columns in both DataFrames.</span>
<span class="sd"> left_on: Column or index level names to join on in the left DataFrame. Can also</span>
<span class="sd"> be an array or list of arrays of the length of the left DataFrame.</span>
<span class="sd"> These arrays are treated as if they are columns.</span>
<span class="sd"> right_on: Column or index level names to join on in the right DataFrame. Can also</span>
<span class="sd"> be an array or list of arrays of the length of the right DataFrame.</span>
<span class="sd"> These arrays are treated as if they are columns.</span>
<span class="sd"> left_index: Use the index from the left DataFrame as the join key(s). If it is a</span>
<span class="sd"> MultiIndex, the number of keys in the other DataFrame (either the index or a number of</span>
<span class="sd"> columns) must match the number of levels.</span>
<span class="sd"> right_index: Use the index from the right DataFrame as the join key. Same caveats as</span>
<span class="sd"> left_index.</span>
<span class="sd"> suffixes: Suffix to apply to overlapping column names in the left and right side,</span>
<span class="sd"> respectively.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> A DataFrame of the two merged objects.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.join : Join columns of another DataFrame.</span>
<span class="sd"> DataFrame.update : Modify in place using non-NA values from another DataFrame.</span>
<span class="sd"> DataFrame.hint : Specifies some hint on the current DataFrame.</span>
<span class="sd"> broadcast : Marks a DataFrame as small enough for use in broadcast joins.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame({&#39;lkey&#39;: [&#39;foo&#39;, &#39;bar&#39;, &#39;baz&#39;, &#39;foo&#39;],</span>
<span class="sd"> ... &#39;value&#39;: [1, 2, 3, 5]},</span>
<span class="sd"> ... columns=[&#39;lkey&#39;, &#39;value&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame({&#39;rkey&#39;: [&#39;foo&#39;, &#39;bar&#39;, &#39;baz&#39;, &#39;foo&#39;],</span>
<span class="sd"> ... &#39;value&#39;: [5, 6, 7, 8]},</span>
<span class="sd"> ... columns=[&#39;rkey&#39;, &#39;value&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df1</span>
<span class="sd"> lkey value</span>
<span class="sd"> 0 foo 1</span>
<span class="sd"> 1 bar 2</span>
<span class="sd"> 2 baz 3</span>
<span class="sd"> 3 foo 5</span>
<span class="sd"> &gt;&gt;&gt; df2</span>
<span class="sd"> rkey value</span>
<span class="sd"> 0 foo 5</span>
<span class="sd"> 1 bar 6</span>
<span class="sd"> 2 baz 7</span>
<span class="sd"> 3 foo 8</span>
<span class="sd"> Merge df1 and df2 on the lkey and rkey columns. The value columns have</span>
<span class="sd"> the default suffixes, _x and _y, appended.</span>
<span class="sd"> &gt;&gt;&gt; merged = df1.merge(df2, left_on=&#39;lkey&#39;, right_on=&#39;rkey&#39;)</span>
<span class="sd"> &gt;&gt;&gt; merged.sort_values(by=[&#39;lkey&#39;, &#39;value_x&#39;, &#39;rkey&#39;, &#39;value_y&#39;]) # doctest: +ELLIPSIS</span>
<span class="sd"> lkey value_x rkey value_y</span>
<span class="sd"> ...bar 2 bar 6</span>
<span class="sd"> ...baz 3 baz 7</span>
<span class="sd"> ...foo 1 foo 5</span>
<span class="sd"> ...foo 1 foo 8</span>
<span class="sd"> ...foo 5 foo 5</span>
<span class="sd"> ...foo 5 foo 8</span>
<span class="sd"> &gt;&gt;&gt; left_psdf = ps.DataFrame({&#39;A&#39;: [1, 2]})</span>
<span class="sd"> &gt;&gt;&gt; right_psdf = ps.DataFrame({&#39;B&#39;: [&#39;x&#39;, &#39;y&#39;]}, index=[1, 2])</span>
<span class="sd"> &gt;&gt;&gt; left_psdf.merge(right_psdf, left_index=True, right_index=True).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 1 2 x</span>
<span class="sd"> &gt;&gt;&gt; left_psdf.merge(right_psdf, left_index=True, right_index=True, how=&#39;left&#39;).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 None</span>
<span class="sd"> 1 2 x</span>
<span class="sd"> &gt;&gt;&gt; left_psdf.merge(right_psdf, left_index=True, right_index=True, how=&#39;right&#39;).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 1 2.0 x</span>
<span class="sd"> 2 NaN y</span>
<span class="sd"> &gt;&gt;&gt; left_psdf.merge(right_psdf, left_index=True, right_index=True, how=&#39;outer&#39;).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1.0 None</span>
<span class="sd"> 1 2.0 x</span>
<span class="sd"> 2 NaN y</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> As described in #263, joining string columns currently returns None for missing values</span>
<span class="sd"> instead of NaN.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">to_list</span><span class="p">(</span><span class="n">os</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]])</span> <span class="o">-&gt;</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]:</span>
<span class="k">if</span> <span class="n">os</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="p">[]</span>
<span class="k">elif</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">os</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span><span class="n">cast</span><span class="p">(</span><span class="n">Label</span><span class="p">,</span> <span class="n">os</span><span class="p">)]</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">os</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[(</span><span class="n">os</span><span class="p">,)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="p">[</span><span class="n">o</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">o</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">o</span><span class="p">,)</span> <span class="k">for</span> <span class="n">o</span> <span class="ow">in</span> <span class="n">os</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">right</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="n">right</span> <span class="o">=</span> <span class="n">right</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="k">if</span> <span class="n">on</span><span class="p">:</span>
<span class="k">if</span> <span class="n">left_on</span> <span class="ow">or</span> <span class="n">right_on</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;Can only pass argument &quot;on&quot; OR &quot;left_on&quot; and &quot;right_on&quot;, &#39;</span>
<span class="s2">&quot;not a combination of both.&quot;</span>
<span class="p">)</span>
<span class="n">left_key_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">,</span> <span class="n">to_list</span><span class="p">(</span><span class="n">on</span><span class="p">)))</span>
<span class="n">right_key_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">,</span> <span class="n">to_list</span><span class="p">(</span><span class="n">on</span><span class="p">)))</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># TODO: need special handling for multi-index.</span>
<span class="k">if</span> <span class="n">left_index</span><span class="p">:</span>
<span class="n">left_key_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">left_key_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">,</span> <span class="n">to_list</span><span class="p">(</span><span class="n">left_on</span><span class="p">)))</span>
<span class="k">if</span> <span class="n">right_index</span><span class="p">:</span>
<span class="n">right_key_names</span> <span class="o">=</span> <span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">right_key_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span>
<span class="nb">map</span><span class="p">(</span><span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">,</span> <span class="n">to_list</span><span class="p">(</span><span class="n">right_on</span><span class="p">))</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">left_key_names</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">right_key_names</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Must pass right_on or right_index=True&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">right_key_names</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">left_key_names</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Must pass left_on or left_index=True&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">left_key_names</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">right_key_names</span><span class="p">:</span>
<span class="n">common</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="n">right</span><span class="o">.</span><span class="n">columns</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">common</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;No common columns to perform merge on. Merge options: &quot;</span>
<span class="s2">&quot;left_on=None, right_on=None, left_index=False, right_index=False&quot;</span>
<span class="p">)</span>
<span class="n">left_key_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">,</span> <span class="n">to_list</span><span class="p">(</span><span class="n">common</span><span class="p">)))</span>
<span class="n">right_key_names</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">,</span> <span class="n">to_list</span><span class="p">(</span><span class="n">common</span><span class="p">)))</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">left_key_names</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">len</span><span class="p">(</span><span class="n">right_key_names</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;len(left_keys) must equal len(right_keys)&quot;</span><span class="p">)</span>
<span class="c1"># We should distinguish the name to avoid ambiguous column name after merging.</span>
<span class="n">right_prefix</span> <span class="o">=</span> <span class="s2">&quot;__right_&quot;</span>
<span class="n">right_key_names</span> <span class="o">=</span> <span class="p">[</span><span class="n">right_prefix</span> <span class="o">+</span> <span class="n">right_key_name</span> <span class="k">for</span> <span class="n">right_key_name</span> <span class="ow">in</span> <span class="n">right_key_names</span><span class="p">]</span>
<span class="n">how</span> <span class="o">=</span> <span class="n">validate_how</span><span class="p">(</span><span class="n">how</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">resolve</span><span class="p">(</span><span class="n">internal</span><span class="p">:</span> <span class="n">InternalFrame</span><span class="p">,</span> <span class="n">side</span><span class="p">:</span> <span class="nb">str</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">InternalFrame</span><span class="p">:</span>
<span class="k">def</span> <span class="nf">rename</span><span class="p">(</span><span class="n">col</span><span class="p">:</span> <span class="nb">str</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">str</span><span class="p">:</span>
<span class="k">return</span> <span class="s2">&quot;__</span><span class="si">{}</span><span class="s2">_</span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">side</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">rename</span><span class="p">(</span><span class="n">col</span><span class="p">))</span>
<span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">sdf</span><span class="o">.</span><span class="n">columns</span>
<span class="k">if</span> <span class="n">col</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">HIDDEN_COLUMNS</span>
<span class="p">],</span>
<span class="o">*</span><span class="n">HIDDEN_COLUMNS</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">rename</span><span class="p">(</span><span class="n">col</span><span class="p">))</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="p">],</span>
<span class="n">index_fields</span><span class="o">=</span><span class="p">[</span>
<span class="n">field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">rename</span><span class="p">(</span><span class="n">field</span><span class="o">.</span><span class="n">name</span><span class="p">))</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="n">internal</span><span class="o">.</span><span class="n">index_fields</span>
<span class="p">],</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">rename</span><span class="p">(</span><span class="n">col</span><span class="p">))</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="p">],</span>
<span class="n">data_fields</span><span class="o">=</span><span class="p">[</span><span class="n">field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">rename</span><span class="p">(</span><span class="n">field</span><span class="o">.</span><span class="n">name</span><span class="p">))</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="n">internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">],</span>
<span class="p">)</span>
<span class="n">left_internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="n">right_internal</span> <span class="o">=</span> <span class="n">resolve</span><span class="p">(</span><span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="p">,</span> <span class="s2">&quot;right&quot;</span><span class="p">)</span>
<span class="n">left_table</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="s2">&quot;left_table&quot;</span><span class="p">)</span>
<span class="n">right_table</span> <span class="o">=</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="s2">&quot;right_table&quot;</span><span class="p">)</span>
<span class="n">left_key_columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">left_table</span><span class="p">,</span> <span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">left_key_names</span><span class="p">]</span>
<span class="n">right_key_columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">right_table</span><span class="p">,</span> <span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">right_key_names</span><span class="p">]</span>
<span class="n">join_condition</span> <span class="o">=</span> <span class="n">reduce</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">:</span> <span class="n">x</span> <span class="o">&amp;</span> <span class="n">y</span><span class="p">,</span>
<span class="p">[</span><span class="n">lkey</span> <span class="o">==</span> <span class="n">rkey</span> <span class="k">for</span> <span class="n">lkey</span><span class="p">,</span> <span class="n">rkey</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">left_key_columns</span><span class="p">,</span> <span class="n">right_key_columns</span><span class="p">)],</span>
<span class="p">)</span>
<span class="n">joined_table</span> <span class="o">=</span> <span class="n">left_table</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">right_table</span><span class="p">,</span> <span class="n">join_condition</span><span class="p">,</span> <span class="n">how</span><span class="o">=</span><span class="n">how</span><span class="p">)</span>
<span class="c1"># Unpack suffixes tuple for convenience</span>
<span class="n">left_suffix</span> <span class="o">=</span> <span class="n">suffixes</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="n">right_suffix</span> <span class="o">=</span> <span class="n">suffixes</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span>
<span class="c1"># Append suffixes to columns with the same name to avoid conflicts later</span>
<span class="n">duplicate_columns</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="n">left_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">&amp;</span> <span class="nb">set</span><span class="p">(</span><span class="n">right_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span>
<span class="n">exprs</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">data_columns</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">def</span> <span class="nf">left_scol_for</span><span class="p">(</span><span class="n">label</span><span class="p">:</span> <span class="n">Label</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">PySparkColumn</span><span class="p">:</span>
<span class="k">return</span> <span class="n">scol_for</span><span class="p">(</span><span class="n">left_table</span><span class="p">,</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="k">def</span> <span class="nf">right_scol_for</span><span class="p">(</span><span class="n">label</span><span class="p">:</span> <span class="n">Label</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">PySparkColumn</span><span class="p">:</span>
<span class="k">return</span> <span class="n">scol_for</span><span class="p">(</span><span class="n">right_table</span><span class="p">,</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">left_scol_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">duplicate_columns</span><span class="p">:</span>
<span class="n">spark_column_name</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span>
<span class="n">spark_column_name</span> <span class="ow">in</span> <span class="n">left_key_names</span>
<span class="ow">and</span> <span class="p">(</span><span class="n">right_prefix</span> <span class="o">+</span> <span class="n">spark_column_name</span><span class="p">)</span> <span class="ow">in</span> <span class="n">right_key_names</span>
<span class="p">):</span>
<span class="n">right_scol</span> <span class="o">=</span> <span class="n">right_scol_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;right&quot;</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">right_scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;full&quot;</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">isNotNull</span><span class="p">(),</span> <span class="n">scol</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">right_scol</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">pass</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">col</span> <span class="o">+</span> <span class="n">left_suffix</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="n">label</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">([</span><span class="nb">str</span><span class="p">(</span><span class="n">label</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> <span class="o">+</span> <span class="n">left_suffix</span><span class="p">]</span> <span class="o">+</span> <span class="nb">list</span><span class="p">(</span><span class="n">label</span><span class="p">[</span><span class="mi">1</span><span class="p">:]))</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol</span><span class="p">)</span>
<span class="n">data_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="c1"># recover `right_prefix` here.</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)[</span><span class="nb">len</span><span class="p">(</span><span class="n">right_prefix</span><span class="p">)</span> <span class="p">:]</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">right_scol_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">duplicate_columns</span><span class="p">:</span>
<span class="n">spark_column_name</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span>
<span class="n">spark_column_name</span> <span class="ow">in</span> <span class="n">left_key_names</span>
<span class="ow">and</span> <span class="p">(</span><span class="n">right_prefix</span> <span class="o">+</span> <span class="n">spark_column_name</span><span class="p">)</span> <span class="ow">in</span> <span class="n">right_key_names</span>
<span class="p">):</span>
<span class="k">continue</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">col</span> <span class="o">+</span> <span class="n">right_suffix</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="n">label</span> <span class="o">=</span> <span class="nb">tuple</span><span class="p">([</span><span class="nb">str</span><span class="p">(</span><span class="n">label</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> <span class="o">+</span> <span class="n">right_suffix</span><span class="p">]</span> <span class="o">+</span> <span class="nb">list</span><span class="p">(</span><span class="n">label</span><span class="p">[</span><span class="mi">1</span><span class="p">:]))</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol</span><span class="p">)</span>
<span class="n">data_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">left_index_scols</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="n">right_index_scols</span> <span class="o">=</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="c1"># Retain indices if they are used for joining</span>
<span class="k">if</span> <span class="n">left_index</span><span class="p">:</span>
<span class="k">if</span> <span class="n">right_index</span><span class="p">:</span>
<span class="k">if</span> <span class="n">how</span> <span class="ow">in</span> <span class="p">(</span><span class="s2">&quot;inner&quot;</span><span class="p">,</span> <span class="s2">&quot;left&quot;</span><span class="p">):</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">left_index_scols</span><span class="p">)</span>
<span class="n">index_spark_column_names</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">index_names</span>
<span class="k">elif</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;right&quot;</span><span class="p">:</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">right_index_scols</span><span class="p">)</span>
<span class="n">index_spark_column_names</span> <span class="o">=</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">index_names</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">index_spark_column_names</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">index_names</span>
<span class="k">for</span> <span class="n">col</span><span class="p">,</span> <span class="n">left_scol</span><span class="p">,</span> <span class="n">right_scol</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span>
<span class="n">index_spark_column_names</span><span class="p">,</span> <span class="n">left_index_scols</span><span class="p">,</span> <span class="n">right_index_scols</span>
<span class="p">):</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">left_scol</span><span class="o">.</span><span class="n">isNotNull</span><span class="p">(),</span> <span class="n">left_scol</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">right_scol</span><span class="p">)</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">right_index_scols</span><span class="p">)</span>
<span class="n">index_spark_column_names</span> <span class="o">=</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">right_internal</span><span class="o">.</span><span class="n">index_names</span>
<span class="k">elif</span> <span class="n">right_index</span><span class="p">:</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">left_index_scols</span><span class="p">)</span>
<span class="n">index_spark_column_names</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">left_internal</span><span class="o">.</span><span class="n">index_names</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">index_spark_column_names</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">selected_columns</span> <span class="o">=</span> <span class="n">joined_table</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">exprs</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">selected_columns</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">selected_columns</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">index_spark_column_names</span>
<span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index_names</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">selected_columns</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">data_columns</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">internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.join"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.join.html#pyspark.pandas.DataFrame.join">[docs]</a> <span class="k">def</span> <span class="nf">join</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">right</span><span class="p">:</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">,</span>
<span class="n">on</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">how</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;left&quot;</span><span class="p">,</span>
<span class="n">lsuffix</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;&quot;</span><span class="p">,</span>
<span class="n">rsuffix</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;&quot;</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Join columns of another DataFrame.</span>
<span class="sd"> Join columns with `right` DataFrame either on index or on a key column. Efficiently join</span>
<span class="sd"> multiple DataFrame objects by index at once by passing a list.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> right: DataFrame, Series</span>
<span class="sd"> on: str, list of str, or array-like, optional</span>
<span class="sd"> Column or index level name(s) in the caller to join on the index in `right`, otherwise</span>
<span class="sd"> joins index-on-index. If multiple values given, the `right` DataFrame must have a</span>
<span class="sd"> MultiIndex. Can pass an array as the join key if it is not already contained in the</span>
<span class="sd"> calling DataFrame. Like an Excel VLOOKUP operation.</span>
<span class="sd"> how: {&#39;left&#39;, &#39;right&#39;, &#39;outer&#39;, &#39;inner&#39;}, default &#39;left&#39;</span>
<span class="sd"> How to handle the operation of the two objects.</span>
<span class="sd"> * left: use `left` frame’s index (or column if on is specified).</span>
<span class="sd"> * right: use `right`’s index.</span>
<span class="sd"> * outer: form union of `left` frame’s index (or column if on is specified) with</span>
<span class="sd"> right’s index, and sort it. lexicographically.</span>
<span class="sd"> * inner: form intersection of `left` frame’s index (or column if on is specified)</span>
<span class="sd"> with `right`’s index, preserving the order of the `left`’s one.</span>
<span class="sd"> lsuffix : str, default &#39;&#39;</span>
<span class="sd"> Suffix to use from left frame&#39;s overlapping columns.</span>
<span class="sd"> rsuffix : str, default &#39;&#39;</span>
<span class="sd"> Suffix to use from `right` frame&#39;s overlapping columns.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> A dataframe containing columns from both the `left` and `right`.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.merge: For column(s)-on-columns(s) operations.</span>
<span class="sd"> DataFrame.update : Modify in place using non-NA values from another DataFrame.</span>
<span class="sd"> DataFrame.hint : Specifies some hint on the current DataFrame.</span>
<span class="sd"> broadcast : Marks a DataFrame as small enough for use in broadcast joins.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> Parameters on, lsuffix, and rsuffix are not supported when passing a list of DataFrame</span>
<span class="sd"> objects.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; psdf1 = ps.DataFrame({&#39;key&#39;: [&#39;K0&#39;, &#39;K1&#39;, &#39;K2&#39;, &#39;K3&#39;],</span>
<span class="sd"> ... &#39;A&#39;: [&#39;A0&#39;, &#39;A1&#39;, &#39;A2&#39;, &#39;A3&#39;]},</span>
<span class="sd"> ... columns=[&#39;key&#39;, &#39;A&#39;])</span>
<span class="sd"> &gt;&gt;&gt; psdf2 = ps.DataFrame({&#39;key&#39;: [&#39;K0&#39;, &#39;K1&#39;, &#39;K2&#39;],</span>
<span class="sd"> ... &#39;B&#39;: [&#39;B0&#39;, &#39;B1&#39;, &#39;B2&#39;]},</span>
<span class="sd"> ... columns=[&#39;key&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; psdf1</span>
<span class="sd"> key A</span>
<span class="sd"> 0 K0 A0</span>
<span class="sd"> 1 K1 A1</span>
<span class="sd"> 2 K2 A2</span>
<span class="sd"> 3 K3 A3</span>
<span class="sd"> &gt;&gt;&gt; psdf2</span>
<span class="sd"> key B</span>
<span class="sd"> 0 K0 B0</span>
<span class="sd"> 1 K1 B1</span>
<span class="sd"> 2 K2 B2</span>
<span class="sd"> Join DataFrames using their indexes.</span>
<span class="sd"> &gt;&gt;&gt; join_psdf = psdf1.join(psdf2, lsuffix=&#39;_left&#39;, rsuffix=&#39;_right&#39;)</span>
<span class="sd"> &gt;&gt;&gt; join_psdf.sort_values(by=join_psdf.columns)</span>
<span class="sd"> key_left A key_right B</span>
<span class="sd"> 0 K0 A0 K0 B0</span>
<span class="sd"> 1 K1 A1 K1 B1</span>
<span class="sd"> 2 K2 A2 K2 B2</span>
<span class="sd"> 3 K3 A3 None None</span>
<span class="sd"> If we want to join using the key columns, we need to set key to be the index in both df and</span>
<span class="sd"> right. The joined DataFrame will have key as its index.</span>
<span class="sd"> &gt;&gt;&gt; join_psdf = psdf1.set_index(&#39;key&#39;).join(psdf2.set_index(&#39;key&#39;))</span>
<span class="sd"> &gt;&gt;&gt; join_psdf.sort_values(by=join_psdf.columns) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> A B</span>
<span class="sd"> key</span>
<span class="sd"> K0 A0 B0</span>
<span class="sd"> K1 A1 B1</span>
<span class="sd"> K2 A2 B2</span>
<span class="sd"> K3 A3 None</span>
<span class="sd"> Another option to join using the key columns is to use the on parameter. DataFrame.join</span>
<span class="sd"> always uses right’s index but we can use any column in df. This method does not preserve</span>
<span class="sd"> the original DataFrame’s index in the result unlike pandas.</span>
<span class="sd"> &gt;&gt;&gt; join_psdf = psdf1.join(psdf2.set_index(&#39;key&#39;), on=&#39;key&#39;)</span>
<span class="sd"> &gt;&gt;&gt; join_psdf.index</span>
<span class="sd"> Index([0, 1, 2, 3], dtype=&#39;int64&#39;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">right</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="n">common</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">intersection</span><span class="p">([</span><span class="n">right</span><span class="o">.</span><span class="n">name</span><span class="p">]))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">common</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="n">right</span><span class="o">.</span><span class="n">columns</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">common</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">lsuffix</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">rsuffix</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;columns overlap but no suffix specified: &quot;</span> <span class="s2">&quot;</span><span class="si">{rename}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">rename</span><span class="o">=</span><span class="n">common</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">need_set_index</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">if</span> <span class="n">on</span><span class="p">:</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">on</span><span class="p">):</span>
<span class="n">on</span> <span class="o">=</span> <span class="p">[</span><span class="n">on</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">on</span><span class="p">)</span> <span class="o">!=</span> <span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s1">&#39;len(left_on) must equal the number of levels in the index of &quot;right&quot;&#39;</span>
<span class="p">)</span>
<span class="n">need_set_index</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">on</span><span class="p">)</span> <span class="o">&amp;</span> <span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="p">))</span> <span class="o">==</span> <span class="mi">0</span>
<span class="k">if</span> <span class="n">need_set_index</span><span class="p">:</span>
<span class="bp">self</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">on</span><span class="p">)</span>
<span class="n">join_psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">merge</span><span class="p">(</span>
<span class="n">right</span><span class="p">,</span> <span class="n">left_index</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">right_index</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">how</span><span class="o">=</span><span class="n">how</span><span class="p">,</span> <span class="n">suffixes</span><span class="o">=</span><span class="p">(</span><span class="n">lsuffix</span><span class="p">,</span> <span class="n">rsuffix</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">join_psdf</span><span class="o">.</span><span class="n">reset_index</span><span class="p">()</span> <span class="k">if</span> <span class="n">need_set_index</span> <span class="k">else</span> <span class="n">join_psdf</span></div>
<div class="viewcode-block" id="DataFrame.combine_first"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.combine_first.html#pyspark.pandas.DataFrame.combine_first">[docs]</a> <span class="k">def</span> <span class="nf">combine_first</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="s2">&quot;DataFrame&quot;</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Update null elements with value in the same location in `other`.</span>
<span class="sd"> Combine two DataFrame objects by filling null values in one DataFrame</span>
<span class="sd"> with non-null values from other DataFrame. The row and column indexes</span>
<span class="sd"> of the resulting DataFrame will be the union of the two.</span>
<span class="sd"> .. versionadded:: 3.3.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> other : DataFrame</span>
<span class="sd"> Provided DataFrame to use to fill null values.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; ps.set_option(&quot;compute.ops_on_diff_frames&quot;, True)</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame({&#39;A&#39;: [None, 0], &#39;B&#39;: [None, 4]})</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame({&#39;A&#39;: [1, 1], &#39;B&#39;: [3, 3]})</span>
<span class="sd"> &gt;&gt;&gt; df1.combine_first(df2).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1.0 3.0</span>
<span class="sd"> 1 0.0 4.0</span>
<span class="sd"> Null values persist if the location of that null value does not exist in other</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame({&#39;A&#39;: [None, 0], &#39;B&#39;: [4, None]})</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame({&#39;B&#39;: [3, 3], &#39;C&#39;: [1, 1]}, index=[1, 2])</span>
<span class="sd"> &gt;&gt;&gt; df1.combine_first(df2).sort_index()</span>
<span class="sd"> A B C</span>
<span class="sd"> 0 NaN 4.0 NaN</span>
<span class="sd"> 1 0.0 3.0 1.0</span>
<span class="sd"> 2 NaN 3.0 1.0</span>
<span class="sd"> &gt;&gt;&gt; ps.reset_option(&quot;compute.ops_on_diff_frames&quot;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;`combine_first` only allows `DataFrame` for parameter `other`&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">same_anchor</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">combined</span> <span class="o">=</span> <span class="bp">self</span>
<span class="n">this</span> <span class="o">=</span> <span class="bp">self</span>
<span class="n">that</span> <span class="o">=</span> <span class="n">other</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">combined</span> <span class="o">=</span> <span class="n">combine_frames</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">this</span> <span class="o">=</span> <span class="n">combined</span><span class="p">[</span><span class="s2">&quot;this&quot;</span><span class="p">]</span>
<span class="n">that</span> <span class="o">=</span> <span class="n">combined</span><span class="p">[</span><span class="s2">&quot;that&quot;</span><span class="p">]</span>
<span class="n">intersect_column_labels</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span>
<span class="nb">set</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">column_labels</span><span class="p">,</span> <span class="n">data_spark_columns</span> <span class="o">=</span> <span class="p">[],</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">column_label</span> <span class="ow">in</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">this_scol</span> <span class="o">=</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">column_label</span> <span class="ow">in</span> <span class="n">intersect_column_labels</span><span class="p">:</span>
<span class="n">that_scol</span> <span class="o">=</span> <span class="n">that</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span>
<span class="n">this_scol_name</span> <span class="o">=</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span>
<span class="n">combined_scol</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">this_scol</span><span class="o">.</span><span class="n">isNull</span><span class="p">(),</span> <span class="n">that_scol</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">this_scol</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">this_scol_name</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">data_spark_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">combined_scol</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">data_spark_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">this_scol</span><span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span>
<span class="k">for</span> <span class="n">column_label</span> <span class="ow">in</span> <span class="n">that</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">column_label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">intersect_column_labels</span><span class="p">:</span>
<span class="n">that_scol</span> <span class="o">=</span> <span class="n">that</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span>
<span class="n">data_spark_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">that_scol</span><span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">combined</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="n">data_spark_columns</span><span class="p">,</span>
<span class="n">data_fields</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="c1"># TODO: dtype?</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</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">internal</span><span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46163): add &#39;filter_func&#39; and &#39;errors&#39; parameter</span>
<div class="viewcode-block" id="DataFrame.update"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.update.html#pyspark.pandas.DataFrame.update">[docs]</a> <span class="k">def</span> <span class="nf">update</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">:</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">,</span> <span class="n">join</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;left&quot;</span><span class="p">,</span> <span class="n">overwrite</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</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;</span>
<span class="sd"> Modify in place using non-NA values from another DataFrame.</span>
<span class="sd"> Aligns on indices. There is no return value.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> other : DataFrame, or Series</span>
<span class="sd"> join : &#39;left&#39;, default &#39;left&#39;</span>
<span class="sd"> Only left join is implemented, keeping the index and columns of the original object.</span>
<span class="sd"> overwrite : bool, default True</span>
<span class="sd"> How to handle non-NA values for overlapping keys:</span>
<span class="sd"> * True: overwrite original DataFrame&#39;s values with values from `other`.</span>
<span class="sd"> * False: only update values that are NA in the original DataFrame.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> None : method directly changes calling object</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.merge : For column(s)-on-columns(s) operations.</span>
<span class="sd"> DataFrame.join : Join columns of another DataFrame.</span>
<span class="sd"> DataFrame.hint : Specifies some hint on the current DataFrame.</span>
<span class="sd"> broadcast : Marks a DataFrame as small enough for use in broadcast joins.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [1, 2, 3], &#39;B&#39;: [400, 500, 600]}, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; new_df = ps.DataFrame({&#39;B&#39;: [4, 5, 6], &#39;C&#39;: [7, 8, 9]}, columns=[&#39;B&#39;, &#39;C&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.update(new_df)</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 4</span>
<span class="sd"> 1 2 5</span>
<span class="sd"> 2 3 6</span>
<span class="sd"> The DataFrame&#39;s length does not increase because of the update,</span>
<span class="sd"> only values at matching index/column labels are updated.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [&#39;a&#39;, &#39;b&#39;, &#39;c&#39;], &#39;B&#39;: [&#39;x&#39;, &#39;y&#39;, &#39;z&#39;]}, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; new_df = ps.DataFrame({&#39;B&#39;: [&#39;d&#39;, &#39;e&#39;, &#39;f&#39;, &#39;g&#39;, &#39;h&#39;, &#39;i&#39;]}, columns=[&#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.update(new_df)</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 a d</span>
<span class="sd"> 1 b e</span>
<span class="sd"> 2 c f</span>
<span class="sd"> For Series, its name attribute must be set.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [&#39;a&#39;, &#39;b&#39;, &#39;c&#39;], &#39;B&#39;: [&#39;x&#39;, &#39;y&#39;, &#39;z&#39;]}, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; new_column = ps.Series([&#39;d&#39;, &#39;e&#39;], name=&#39;B&#39;, index=[0, 2])</span>
<span class="sd"> &gt;&gt;&gt; df.update(new_column)</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 a d</span>
<span class="sd"> 1 b y</span>
<span class="sd"> 2 c e</span>
<span class="sd"> If `other` contains None the corresponding values are not updated in the original dataframe.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [1, 2, 3], &#39;B&#39;: [400, 500, 600]}, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; new_df = ps.DataFrame({&#39;B&#39;: [4, None, 6]}, columns=[&#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.update(new_df)</span>
<span class="sd"> &gt;&gt;&gt; df.sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 4.0</span>
<span class="sd"> 1 2 500.0</span>
<span class="sd"> 2 3 6.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">join</span> <span class="o">!=</span> <span class="s2">&quot;left&quot;</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;Only left join is supported&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="n">other</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="n">update_columns</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span>
<span class="nb">set</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span><span class="o">.</span><span class="n">intersection</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">))</span>
<span class="p">)</span>
<span class="n">update_sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">join</span><span class="p">(</span>
<span class="n">other</span><span class="p">[</span><span class="n">update_columns</span><span class="p">],</span> <span class="n">rsuffix</span><span class="o">=</span><span class="s2">&quot;_new&quot;</span>
<span class="p">)</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="k">for</span> <span class="n">column_labels</span> <span class="ow">in</span> <span class="n">update_columns</span><span class="p">:</span>
<span class="n">column_name</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span>
<span class="n">old_col</span> <span class="o">=</span> <span class="n">scol_for</span><span class="p">(</span><span class="n">update_sdf</span><span class="p">,</span> <span class="n">column_name</span><span class="p">)</span>
<span class="n">new_col</span> <span class="o">=</span> <span class="n">scol_for</span><span class="p">(</span>
<span class="n">update_sdf</span><span class="p">,</span> <span class="n">other</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">+</span> <span class="s2">&quot;_new&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">overwrite</span><span class="p">:</span>
<span class="n">update_sdf</span> <span class="o">=</span> <span class="n">update_sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">column_name</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">new_col</span><span class="o">.</span><span class="n">isNull</span><span class="p">(),</span> <span class="n">old_col</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">new_col</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">update_sdf</span> <span class="o">=</span> <span class="n">update_sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">column_name</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">old_col</span><span class="o">.</span><span class="n">isNull</span><span class="p">(),</span> <span class="n">new_col</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">old_col</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">data_fields</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)]</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">update_sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">update_sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_names</span><span class="p">],</span>
<span class="o">*</span><span class="n">HIDDEN_COLUMNS</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">data_fields</span><span class="o">=</span><span class="n">data_fields</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">internal</span><span class="p">,</span> <span class="n">check_same_anchor</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.cov"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.cov.html#pyspark.pandas.DataFrame.cov">[docs]</a> <span class="k">def</span> <span class="nf">cov</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">min_periods</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> <span class="n">ddof</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="mi">1</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute pairwise covariance of columns, excluding NA/null values.</span>
<span class="sd"> Compute the pairwise covariance among the series of a DataFrame.</span>
<span class="sd"> The returned data frame is the `covariance matrix</span>
<span class="sd"> &lt;https://en.wikipedia.org/wiki/Covariance_matrix&gt;`__ of the columns</span>
<span class="sd"> of the DataFrame.</span>
<span class="sd"> Both NA and null values are automatically excluded from the</span>
<span class="sd"> calculation. (See the note below about bias from missing values.)</span>
<span class="sd"> A threshold can be set for the minimum number of</span>
<span class="sd"> observations for each value created. Comparisons with observations</span>
<span class="sd"> below this threshold will be returned as ``NaN``.</span>
<span class="sd"> This method is generally used for the analysis of time series data to</span>
<span class="sd"> understand the relationship between different measures across time.</span>
<span class="sd"> .. versionadded:: 3.3.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> min_periods : int, optional</span>
<span class="sd"> Minimum number of observations required per pair of columns</span>
<span class="sd"> to have a valid result.</span>
<span class="sd"> ddof : int, default 1</span>
<span class="sd"> Delta degrees of freedom. The divisor used in calculations</span>
<span class="sd"> is ``N - ddof``, where ``N`` represents the number of elements.</span>
<span class="sd"> .. versionadded:: 3.4.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> The covariance matrix of the series of the DataFrame.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.cov : Compute covariance with another Series.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(1, 2), (0, 3), (2, 0), (1, 1)],</span>
<span class="sd"> ... columns=[&#39;dogs&#39;, &#39;cats&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.cov()</span>
<span class="sd"> dogs cats</span>
<span class="sd"> dogs 0.666667 -1.000000</span>
<span class="sd"> cats -1.000000 1.666667</span>
<span class="sd"> &gt;&gt;&gt; np.random.seed(42)</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(np.random.randn(1000, 5),</span>
<span class="sd"> ... columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;, &#39;d&#39;, &#39;e&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.cov()</span>
<span class="sd"> a b c d e</span>
<span class="sd"> a 0.998438 -0.020161 0.059277 -0.008943 0.014144</span>
<span class="sd"> b -0.020161 1.059352 -0.008543 -0.024738 0.009826</span>
<span class="sd"> c 0.059277 -0.008543 1.010670 -0.001486 -0.000271</span>
<span class="sd"> d -0.008943 -0.024738 -0.001486 0.921297 -0.013692</span>
<span class="sd"> e 0.014144 0.009826 -0.000271 -0.013692 0.977795</span>
<span class="sd"> &gt;&gt;&gt; df.cov(ddof=2)</span>
<span class="sd"> a b c d e</span>
<span class="sd"> a 0.999439 -0.020181 0.059336 -0.008952 0.014159</span>
<span class="sd"> b -0.020181 1.060413 -0.008551 -0.024762 0.009836</span>
<span class="sd"> c 0.059336 -0.008551 1.011683 -0.001487 -0.000271</span>
<span class="sd"> d -0.008952 -0.024762 -0.001487 0.922220 -0.013705</span>
<span class="sd"> e 0.014159 0.009836 -0.000271 -0.013705 0.978775</span>
<span class="sd"> &gt;&gt;&gt; df.cov(ddof=-1)</span>
<span class="sd"> a b c d e</span>
<span class="sd"> a 0.996444 -0.020121 0.059158 -0.008926 0.014116</span>
<span class="sd"> b -0.020121 1.057235 -0.008526 -0.024688 0.009807</span>
<span class="sd"> c 0.059158 -0.008526 1.008650 -0.001483 -0.000270</span>
<span class="sd"> d -0.008926 -0.024688 -0.001483 0.919456 -0.013664</span>
<span class="sd"> e 0.014116 0.009807 -0.000270 -0.013664 0.975842</span>
<span class="sd"> **Minimum number of periods**</span>
<span class="sd"> This method also supports an optional ``min_periods`` keyword</span>
<span class="sd"> that specifies the required minimum number of non-NA observations for</span>
<span class="sd"> each column pair to have a valid result:</span>
<span class="sd"> &gt;&gt;&gt; np.random.seed(42)</span>
<span class="sd"> &gt;&gt;&gt; df = pd.DataFrame(np.random.randn(20, 3),</span>
<span class="sd"> ... columns=[&#39;a&#39;, &#39;b&#39;, &#39;c&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.loc[df.index[:5], &#39;a&#39;] = np.nan</span>
<span class="sd"> &gt;&gt;&gt; df.loc[df.index[5:10], &#39;b&#39;] = np.nan</span>
<span class="sd"> &gt;&gt;&gt; sdf = ps.from_pandas(df)</span>
<span class="sd"> &gt;&gt;&gt; sdf.cov(min_periods=12)</span>
<span class="sd"> a b c</span>
<span class="sd"> a 0.316741 NaN -0.150812</span>
<span class="sd"> b NaN 1.248003 0.191417</span>
<span class="sd"> c -0.150812 0.191417 0.895202</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ddof</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;ddof must be integer&quot;</span><span class="p">)</span>
<span class="n">min_periods</span> <span class="o">=</span> <span class="mi">1</span> <span class="k">if</span> <span class="n">min_periods</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="n">min_periods</span>
<span class="c1"># Only compute covariance for Boolean and Numeric except Decimal</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span>
<span class="p">[</span>
<span class="n">col</span>
<span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">[</span><span class="n">col</span><span class="p">]</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">)</span>
<span class="ow">or</span> <span class="p">(</span>
<span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">[</span><span class="n">col</span><span class="p">]</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="n">NumericType</span><span class="p">)</span>
<span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="p">[</span><span class="n">col</span><span class="p">]</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="n">DecimalType</span><span class="p">)</span>
<span class="p">)</span>
<span class="p">]</span>
<span class="p">]</span>
<span class="n">num_cols</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span>
<span class="n">cov</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">zeros</span><span class="p">([</span><span class="n">num_cols</span><span class="p">,</span> <span class="n">num_cols</span><span class="p">])</span>
<span class="k">if</span> <span class="n">num_cols</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">()</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">psdf</span><span class="p">)</span> <span class="o">&lt;</span> <span class="n">min_periods</span><span class="p">:</span>
<span class="n">cov</span><span class="o">.</span><span class="n">fill</span><span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">nan</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">cov</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span>
<span class="n">data_cols</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="n">cov_scols</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">count_not_null_scols</span> <span class="o">=</span> <span class="p">[]</span>
<span class="c1"># Count number of null row between two columns</span>
<span class="c1"># Example:</span>
<span class="c1"># a b c</span>
<span class="c1"># 0 1 1 1</span>
<span class="c1"># 1 NaN 2 2</span>
<span class="c1"># 2 3 NaN 3</span>
<span class="c1"># 3 4 4 4</span>
<span class="c1">#</span>
<span class="c1"># a b c</span>
<span class="c1"># a count(a, a) count(a, b) count(a, c)</span>
<span class="c1"># b count(b, b) count(b, c)</span>
<span class="c1"># c count(c, c)</span>
<span class="c1">#</span>
<span class="c1"># count_not_null_scols =</span>
<span class="c1"># [F.count(a, a), F.count(a, b), F.count(a, c), F.count(b, b), F.count(b, c), F.count(c, c)]</span>
<span class="k">for</span> <span class="n">r</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">num_cols</span><span class="p">):</span>
<span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">r</span><span class="p">,</span> <span class="n">num_cols</span><span class="p">):</span>
<span class="n">count_not_null_scols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">count</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">data_cols</span><span class="p">[</span><span class="n">r</span><span class="p">])</span><span class="o">.</span><span class="n">isNotNull</span><span class="p">()</span> <span class="o">&amp;</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">data_cols</span><span class="p">[</span><span class="n">c</span><span class="p">])</span><span class="o">.</span><span class="n">isNotNull</span><span class="p">(),</span> <span class="mi">1</span><span class="p">)</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">count_not_null</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span><span class="nb">float</span><span class="p">(</span><span class="s2">&quot;nan&quot;</span><span class="p">),</span> <span class="kc">None</span><span class="p">)</span>
<span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">count_not_null_scols</span><span class="p">)</span>
<span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="mi">1</span><span class="p">)[</span><span class="mi">0</span><span class="p">]</span>
<span class="p">)</span>
<span class="c1"># Calculate covariance between two columns</span>
<span class="c1"># Example:</span>
<span class="c1"># with min_periods = 3</span>
<span class="c1"># a b c</span>
<span class="c1"># 0 1 1 1</span>
<span class="c1"># 1 NaN 2 2</span>
<span class="c1"># 2 3 NaN 3</span>
<span class="c1"># 3 4 4 4</span>
<span class="c1">#</span>
<span class="c1"># a b c</span>
<span class="c1"># a cov(a, a) None cov(a, c)</span>
<span class="c1"># b cov(b, b) cov(b, c)</span>
<span class="c1"># c cov(c, c)</span>
<span class="c1">#</span>
<span class="c1"># cov_scols = [F.cov(a, a), None, F.cov(a, c), F.cov(b, b), F.cov(b, c), F.cov(c, c)]</span>
<span class="n">step</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">for</span> <span class="n">r</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">num_cols</span><span class="p">):</span>
<span class="n">step</span> <span class="o">+=</span> <span class="n">r</span>
<span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">r</span><span class="p">,</span> <span class="n">num_cols</span><span class="p">):</span>
<span class="n">cov_scols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">SF</span><span class="o">.</span><span class="n">covar</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">data_cols</span><span class="p">[</span><span class="n">r</span><span class="p">])</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;double&quot;</span><span class="p">),</span> <span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">data_cols</span><span class="p">[</span><span class="n">c</span><span class="p">])</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;double&quot;</span><span class="p">),</span> <span class="n">ddof</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">count_not_null</span><span class="p">[</span><span class="n">r</span> <span class="o">*</span> <span class="n">num_cols</span> <span class="o">+</span> <span class="n">c</span> <span class="o">-</span> <span class="n">step</span><span class="p">]</span> <span class="o">&gt;=</span> <span class="n">min_periods</span>
<span class="k">else</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">pair_cov</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">cov_scols</span><span class="p">)</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="mi">1</span><span class="p">)[</span><span class="mi">0</span><span class="p">]</span>
<span class="c1"># Convert from row to 2D array</span>
<span class="c1"># Example:</span>
<span class="c1"># pair_cov = [cov(a, a), None, cov(a, c), cov(b, b), cov(b, c), cov(c, c)]</span>
<span class="c1">#</span>
<span class="c1"># cov =</span>
<span class="c1">#</span>
<span class="c1"># a b c</span>
<span class="c1"># a cov(a, a) None cov(a, c)</span>
<span class="c1"># b cov(b, b) cov(b, c)</span>
<span class="c1"># c cov(c, c)</span>
<span class="n">step</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">for</span> <span class="n">r</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">num_cols</span><span class="p">):</span>
<span class="n">step</span> <span class="o">+=</span> <span class="n">r</span>
<span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">r</span><span class="p">,</span> <span class="n">num_cols</span><span class="p">):</span>
<span class="n">cov</span><span class="p">[</span><span class="n">r</span><span class="p">][</span><span class="n">c</span><span class="p">]</span> <span class="o">=</span> <span class="n">pair_cov</span><span class="p">[</span><span class="n">r</span> <span class="o">*</span> <span class="n">num_cols</span> <span class="o">+</span> <span class="n">c</span> <span class="o">-</span> <span class="n">step</span><span class="p">]</span>
<span class="c1"># Copy values</span>
<span class="c1"># Example:</span>
<span class="c1"># cov =</span>
<span class="c1"># a b c</span>
<span class="c1"># a cov(a, a) None cov(a, c)</span>
<span class="c1"># b None cov(b, b) cov(b, c)</span>
<span class="c1"># c cov(a, c) cov(b, c) cov(c, c)</span>
<span class="n">cov</span> <span class="o">=</span> <span class="n">cov</span> <span class="o">+</span> <span class="n">cov</span><span class="o">.</span><span class="n">T</span> <span class="o">-</span> <span class="n">np</span><span class="o">.</span><span class="n">diag</span><span class="p">(</span><span class="n">np</span><span class="o">.</span><span class="n">diag</span><span class="p">(</span><span class="n">cov</span><span class="p">))</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">cov</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.sample"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.sample.html#pyspark.pandas.DataFrame.sample">[docs]</a> <span class="k">def</span> <span class="nf">sample</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">n</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">frac</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">float</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">replace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">random_state</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">ignore_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a random sample of items from an axis of object.</span>
<span class="sd"> Please call this function using named argument by specifying the ``frac`` argument.</span>
<span class="sd"> You can use `random_state` for reproducibility. However, note that different from pandas,</span>
<span class="sd"> specifying a seed in pandas-on-Spark/Spark does not guarantee the sampled rows will</span>
<span class="sd"> be fixed. The result set depends on not only the seed, but also how the data is distributed</span>
<span class="sd"> across machines and to some extent network randomness when shuffle operations are involved.</span>
<span class="sd"> Even in the simplest case, the result set will depend on the system&#39;s CPU core count.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> n : int, optional</span>
<span class="sd"> Number of items to return. This is currently NOT supported. Use frac instead.</span>
<span class="sd"> frac : float, optional</span>
<span class="sd"> Fraction of axis items to return.</span>
<span class="sd"> replace : bool, default False</span>
<span class="sd"> Sample with or without replacement.</span>
<span class="sd"> random_state : int, optional</span>
<span class="sd"> Seed for the random number generator (if int).</span>
<span class="sd"> ignore_index : bool, default False</span>
<span class="sd"> If True, the resulting index will be labeled 0, 1, …, n - 1.</span>
<span class="sd"> .. versionadded:: 3.4.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series or DataFrame</span>
<span class="sd"> A new object of same type as caller containing the sampled items.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;num_legs&#39;: [2, 4, 8, 0],</span>
<span class="sd"> ... &#39;num_wings&#39;: [2, 0, 0, 0],</span>
<span class="sd"> ... &#39;num_specimen_seen&#39;: [10, 2, 1, 8]},</span>
<span class="sd"> ... index=[&#39;falcon&#39;, &#39;dog&#39;, &#39;spider&#39;, &#39;fish&#39;],</span>
<span class="sd"> ... columns=[&#39;num_legs&#39;, &#39;num_wings&#39;, &#39;num_specimen_seen&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +SKIP</span>
<span class="sd"> num_legs num_wings num_specimen_seen</span>
<span class="sd"> falcon 2 2 10</span>
<span class="sd"> dog 4 0 2</span>
<span class="sd"> spider 8 0 1</span>
<span class="sd"> fish 0 0 8</span>
<span class="sd"> A random 25% sample of the ``DataFrame``.</span>
<span class="sd"> Note that we use `random_state` to ensure the reproducibility of</span>
<span class="sd"> the examples.</span>
<span class="sd"> &gt;&gt;&gt; df.sample(frac=0.25, random_state=1) # doctest: +SKIP</span>
<span class="sd"> num_legs num_wings num_specimen_seen</span>
<span class="sd"> falcon 2 2 10</span>
<span class="sd"> fish 0 0 8</span>
<span class="sd"> A random 50% sample of the ``DataFrame``, while ignoring the index.</span>
<span class="sd"> &gt;&gt;&gt; df.sample(frac=0.5, random_state=1, ignore_index=True) # doctest: +SKIP</span>
<span class="sd"> num_legs num_wings num_specimen_seen</span>
<span class="sd"> 0 4 0 2</span>
<span class="sd"> 1 8 0 1</span>
<span class="sd"> 2 0 0 8</span>
<span class="sd"> Extract 25% random elements from the ``Series`` ``df[&#39;num_legs&#39;]`` with replacement</span>
<span class="sd"> so, the same items could appear more than once.</span>
<span class="sd"> &gt;&gt;&gt; df[&#39;num_legs&#39;].sample(frac=0.4, replace=True, random_state=1) # doctest: +SKIP</span>
<span class="sd"> falcon 2</span>
<span class="sd"> spider 8</span>
<span class="sd"> spider 8</span>
<span class="sd"> Name: num_legs, dtype: int64</span>
<span class="sd"> Specifying the exact number of items to return is not supported now.</span>
<span class="sd"> &gt;&gt;&gt; df.sample(n=5) # doctest: +ELLIPSIS</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> NotImplementedError: Function sample currently does not support specifying ...</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Note: we don&#39;t run any of the doctests because the result can change depending on the</span>
<span class="c1"># system&#39;s core count.</span>
<span class="k">if</span> <span class="n">n</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span>
<span class="s2">&quot;Function sample currently does not support specifying &quot;</span>
<span class="s2">&quot;exact number of items to return. Use frac instead.&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">frac</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;frac must be specified.&quot;</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">sample</span><span class="p">(</span>
<span class="n">withReplacement</span><span class="o">=</span><span class="n">replace</span><span class="p">,</span> <span class="n">fraction</span><span class="o">=</span><span class="n">frac</span><span class="p">,</span> <span class="n">seed</span><span class="o">=</span><span class="n">random_state</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">ignore_index</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">sdf</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="o">*</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">sdf</span><span class="p">))</span></div>
<div class="viewcode-block" id="DataFrame.astype"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.astype.html#pyspark.pandas.DataFrame.astype">[docs]</a> <span class="k">def</span> <span class="nf">astype</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">dtype</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">]]])</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Cast a pandas-on-Spark object to a specified dtype ``dtype``.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> dtype : data type, or dict of column name -&gt; data type</span>
<span class="sd"> Use a numpy.dtype or Python type to cast entire pandas-on-Spark object to</span>
<span class="sd"> the same type. Alternatively, use {col: dtype, ...}, where col is a</span>
<span class="sd"> column label and dtype is a numpy.dtype or Python type to cast one</span>
<span class="sd"> or more of the DataFrame&#39;s columns to column-specific types.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> casted : same type as caller</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> to_datetime : Convert argument to datetime.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;a&#39;: [1, 2, 3], &#39;b&#39;: [1, 2, 3]}, dtype=&#39;int64&#39;)</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 1</span>
<span class="sd"> 1 2 2</span>
<span class="sd"> 2 3 3</span>
<span class="sd"> Convert to float type:</span>
<span class="sd"> &gt;&gt;&gt; df.astype(&#39;float&#39;)</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1.0 1.0</span>
<span class="sd"> 1 2.0 2.0</span>
<span class="sd"> 2 3.0 3.0</span>
<span class="sd"> Convert to int64 type back:</span>
<span class="sd"> &gt;&gt;&gt; df.astype(&#39;int64&#39;)</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 1</span>
<span class="sd"> 1 2 2</span>
<span class="sd"> 2 3 3</span>
<span class="sd"> Convert column a to float type:</span>
<span class="sd"> &gt;&gt;&gt; df.astype({&#39;a&#39;: float})</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1.0 1</span>
<span class="sd"> 1 2.0 2</span>
<span class="sd"> 2 3.0 3</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">applied</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">if</span> <span class="n">is_dict_like</span><span class="p">(</span><span class="n">dtype</span><span class="p">):</span>
<span class="n">dtype_dict</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">]],</span> <span class="n">dtype</span><span class="p">)</span>
<span class="k">for</span> <span class="n">col_name</span> <span class="ow">in</span> <span class="n">dtype_dict</span><span class="o">.</span><span class="n">keys</span><span class="p">():</span>
<span class="k">if</span> <span class="n">col_name</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span>
<span class="s2">&quot;Only a column name can be used for the &quot;</span>
<span class="s2">&quot;key in a dtype mappings argument.&quot;</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">col_name</span><span class="p">,</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">if</span> <span class="n">col_name</span> <span class="ow">in</span> <span class="n">dtype_dict</span><span class="p">:</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">col</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span><span class="n">dtype</span><span class="o">=</span><span class="n">dtype_dict</span><span class="p">[</span><span class="n">col_name</span><span class="p">]))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">for</span> <span class="n">col_name</span><span class="p">,</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">col</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span><span class="n">dtype</span><span class="o">=</span><span class="n">cast</span><span class="p">(</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">],</span> <span class="n">dtype</span><span class="p">)))</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">applied</span><span class="p">))</span></div>
<div class="viewcode-block" id="DataFrame.add_prefix"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.add_prefix.html#pyspark.pandas.DataFrame.add_prefix">[docs]</a> <span class="k">def</span> <span class="nf">add_prefix</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">prefix</span><span class="p">:</span> <span class="nb">str</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Prefix labels with string `prefix`.</span>
<span class="sd"> For Series, the row labels are prefixed.</span>
<span class="sd"> For DataFrame, the column labels are prefixed.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> prefix : str</span>
<span class="sd"> The string to add before each label.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> New DataFrame with updated labels.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.add_prefix: Prefix row labels with string `prefix`.</span>
<span class="sd"> Series.add_suffix: Suffix row labels with string `suffix`.</span>
<span class="sd"> DataFrame.add_suffix: Suffix column labels with string `suffix`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [1, 2, 3, 4], &#39;B&#39;: [3, 4, 5, 6]}, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 3</span>
<span class="sd"> 1 2 4</span>
<span class="sd"> 2 3 5</span>
<span class="sd"> 3 4 6</span>
<span class="sd"> &gt;&gt;&gt; df.add_prefix(&#39;col_&#39;)</span>
<span class="sd"> col_A col_B</span>
<span class="sd"> 0 1 3</span>
<span class="sd"> 1 2 4</span>
<span class="sd"> 2 3 5</span>
<span class="sd"> 3 4 6</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">prefix</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="nb">tuple</span><span class="p">([</span><span class="n">prefix</span> <span class="o">+</span> <span class="n">i</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">psser</span><span class="o">.</span><span class="n">_column_label</span><span class="p">]))</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.add_suffix"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.add_suffix.html#pyspark.pandas.DataFrame.add_suffix">[docs]</a> <span class="k">def</span> <span class="nf">add_suffix</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">suffix</span><span class="p">:</span> <span class="nb">str</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Suffix labels with string `suffix`.</span>
<span class="sd"> For Series, the row labels are suffixed.</span>
<span class="sd"> For DataFrame, the column labels are suffixed.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> suffix : str</span>
<span class="sd"> The string to add before each label.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> New DataFrame with updated labels.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.add_prefix: Prefix row labels with string `prefix`.</span>
<span class="sd"> Series.add_suffix: Suffix row labels with string `suffix`.</span>
<span class="sd"> DataFrame.add_prefix: Prefix column labels with string `prefix`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [1, 2, 3, 4], &#39;B&#39;: [3, 4, 5, 6]}, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 3</span>
<span class="sd"> 1 2 4</span>
<span class="sd"> 2 3 5</span>
<span class="sd"> 3 4 6</span>
<span class="sd"> &gt;&gt;&gt; df.add_suffix(&#39;_col&#39;)</span>
<span class="sd"> A_col B_col</span>
<span class="sd"> 0 1 3</span>
<span class="sd"> 1 2 4</span>
<span class="sd"> 2 3 5</span>
<span class="sd"> 3 4 6</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">suffix</span><span class="p">,</span> <span class="nb">str</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="nb">tuple</span><span class="p">([</span><span class="n">i</span> <span class="o">+</span> <span class="n">suffix</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">psser</span><span class="o">.</span><span class="n">_column_label</span><span class="p">]))</span>
<span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46164): include and exclude should be implemented.</span>
<div class="viewcode-block" id="DataFrame.describe"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.describe.html#pyspark.pandas.DataFrame.describe">[docs]</a> <span class="k">def</span> <span class="nf">describe</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">percentiles</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="nb">float</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="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Generate descriptive statistics that summarize the central tendency,</span>
<span class="sd"> dispersion and shape of a dataset&#39;s distribution, excluding</span>
<span class="sd"> ``NaN`` values.</span>
<span class="sd"> Analyzes both numeric and object series, as well</span>
<span class="sd"> as ``DataFrame`` column sets of mixed data types. The output</span>
<span class="sd"> will vary depending on what is provided. Refer to the notes</span>
<span class="sd"> below for more detail.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> percentiles : list of ``float`` in range [0.0, 1.0], default [0.25, 0.5, 0.75]</span>
<span class="sd"> A list of percentiles to be computed.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Summary statistics of the Dataframe provided.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.count: Count number of non-NA/null observations.</span>
<span class="sd"> DataFrame.max: Maximum of the values in the object.</span>
<span class="sd"> DataFrame.min: Minimum of the values in the object.</span>
<span class="sd"> DataFrame.mean: Mean of the values.</span>
<span class="sd"> DataFrame.std: Standard deviation of the observations.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> For numeric data, the result&#39;s index will include ``count``,</span>
<span class="sd"> ``mean``, ``std``, ``min``, ``25%``, ``50%``, ``75%``, ``max``.</span>
<span class="sd"> For object data (e.g. strings or timestamps), the result’s index will include</span>
<span class="sd"> ``count``, ``unique``, ``top``, and ``freq``.</span>
<span class="sd"> The ``top`` is the most common value. The ``freq`` is the most common value’s frequency.</span>
<span class="sd"> Timestamps also include the ``first`` and ``last`` items.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> Describing a numeric ``Series``.</span>
<span class="sd"> &gt;&gt;&gt; s = ps.Series([1, 2, 3])</span>
<span class="sd"> &gt;&gt;&gt; s.describe()</span>
<span class="sd"> count 3.0</span>
<span class="sd"> mean 2.0</span>
<span class="sd"> std 1.0</span>
<span class="sd"> min 1.0</span>
<span class="sd"> 25% 1.0</span>
<span class="sd"> 50% 2.0</span>
<span class="sd"> 75% 3.0</span>
<span class="sd"> max 3.0</span>
<span class="sd"> dtype: float64</span>
<span class="sd"> Describing a ``DataFrame``. Only numeric fields are returned.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;numeric1&#39;: [1, 2, 3],</span>
<span class="sd"> ... &#39;numeric2&#39;: [4.0, 5.0, 6.0],</span>
<span class="sd"> ... &#39;object&#39;: [&#39;a&#39;, &#39;b&#39;, &#39;c&#39;]</span>
<span class="sd"> ... },</span>
<span class="sd"> ... columns=[&#39;numeric1&#39;, &#39;numeric2&#39;, &#39;object&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.describe()</span>
<span class="sd"> numeric1 numeric2</span>
<span class="sd"> count 3.0 3.0</span>
<span class="sd"> mean 2.0 5.0</span>
<span class="sd"> std 1.0 1.0</span>
<span class="sd"> min 1.0 4.0</span>
<span class="sd"> 25% 1.0 4.0</span>
<span class="sd"> 50% 2.0 5.0</span>
<span class="sd"> 75% 3.0 6.0</span>
<span class="sd"> max 3.0 6.0</span>
<span class="sd"> For multi-index columns:</span>
<span class="sd"> &gt;&gt;&gt; df.columns = [(&#39;num&#39;, &#39;a&#39;), (&#39;num&#39;, &#39;b&#39;), (&#39;obj&#39;, &#39;c&#39;)]</span>
<span class="sd"> &gt;&gt;&gt; df.describe() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num</span>
<span class="sd"> a b</span>
<span class="sd"> count 3.0 3.0</span>
<span class="sd"> mean 2.0 5.0</span>
<span class="sd"> std 1.0 1.0</span>
<span class="sd"> min 1.0 4.0</span>
<span class="sd"> 25% 1.0 4.0</span>
<span class="sd"> 50% 2.0 5.0</span>
<span class="sd"> 75% 3.0 6.0</span>
<span class="sd"> max 3.0 6.0</span>
<span class="sd"> &gt;&gt;&gt; df[(&#39;num&#39;, &#39;b&#39;)].describe()</span>
<span class="sd"> count 3.0</span>
<span class="sd"> mean 5.0</span>
<span class="sd"> std 1.0</span>
<span class="sd"> min 4.0</span>
<span class="sd"> 25% 4.0</span>
<span class="sd"> 50% 5.0</span>
<span class="sd"> 75% 6.0</span>
<span class="sd"> max 6.0</span>
<span class="sd"> Name: (num, b), dtype: float64</span>
<span class="sd"> Describing a ``DataFrame`` and selecting custom percentiles.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;numeric1&#39;: [1, 2, 3],</span>
<span class="sd"> ... &#39;numeric2&#39;: [4.0, 5.0, 6.0]</span>
<span class="sd"> ... },</span>
<span class="sd"> ... columns=[&#39;numeric1&#39;, &#39;numeric2&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df.describe(percentiles = [0.85, 0.15])</span>
<span class="sd"> numeric1 numeric2</span>
<span class="sd"> count 3.0 3.0</span>
<span class="sd"> mean 2.0 5.0</span>
<span class="sd"> std 1.0 1.0</span>
<span class="sd"> min 1.0 4.0</span>
<span class="sd"> 15% 1.0 4.0</span>
<span class="sd"> 50% 2.0 5.0</span>
<span class="sd"> 85% 3.0 6.0</span>
<span class="sd"> max 3.0 6.0</span>
<span class="sd"> Describing a column from a ``DataFrame`` by accessing it as</span>
<span class="sd"> an attribute.</span>
<span class="sd"> &gt;&gt;&gt; df.numeric1.describe()</span>
<span class="sd"> count 3.0</span>
<span class="sd"> mean 2.0</span>
<span class="sd"> std 1.0</span>
<span class="sd"> min 1.0</span>
<span class="sd"> 25% 1.0</span>
<span class="sd"> 50% 2.0</span>
<span class="sd"> 75% 3.0</span>
<span class="sd"> max 3.0</span>
<span class="sd"> Name: numeric1, dtype: float64</span>
<span class="sd"> Describing a column from a ``DataFrame`` by accessing it as</span>
<span class="sd"> an attribute and selecting custom percentiles.</span>
<span class="sd"> &gt;&gt;&gt; df.numeric1.describe(percentiles = [0.85, 0.15])</span>
<span class="sd"> count 3.0</span>
<span class="sd"> mean 2.0</span>
<span class="sd"> std 1.0</span>
<span class="sd"> min 1.0</span>
<span class="sd"> 15% 1.0</span>
<span class="sd"> 50% 2.0</span>
<span class="sd"> 85% 3.0</span>
<span class="sd"> max 3.0</span>
<span class="sd"> Name: numeric1, dtype: float64</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">psser_numeric</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Series</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">psser_string</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Series</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">psser_timestamp</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Series</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">spark_data_types</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">DataType</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">column_labels</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">column_names</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">spark_data_type</span> <span class="o">=</span> <span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">spark_data_type</span><span class="p">,</span> <span class="n">NumericType</span><span class="p">):</span>
<span class="n">psser_numeric</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">psser</span><span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">spark_data_types</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">spark_data_type</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">spark_data_type</span><span class="p">,</span> <span class="p">(</span><span class="n">TimestampType</span><span class="p">,</span> <span class="n">TimestampNTZType</span><span class="p">)):</span>
<span class="n">psser_timestamp</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">psser</span><span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">spark_data_types</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">spark_data_type</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">psser_string</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">psser</span><span class="p">)</span>
<span class="n">column_names</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="k">if</span> <span class="n">percentiles</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">((</span><span class="n">p</span> <span class="o">&lt;</span> <span class="mf">0.0</span><span class="p">)</span> <span class="ow">or</span> <span class="p">(</span><span class="n">p</span> <span class="o">&gt;</span> <span class="mf">1.0</span><span class="p">)</span> <span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="n">percentiles</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Percentiles should all be in the interval [0, 1]&quot;</span><span class="p">)</span>
<span class="c1"># appending 50% if not in percentiles already</span>
<span class="n">percentiles</span> <span class="o">=</span> <span class="p">(</span><span class="n">percentiles</span> <span class="o">+</span> <span class="p">[</span><span class="mf">0.5</span><span class="p">])</span> <span class="k">if</span> <span class="mf">0.5</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">percentiles</span> <span class="k">else</span> <span class="n">percentiles</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">percentiles</span> <span class="o">=</span> <span class="p">[</span><span class="mf">0.25</span><span class="p">,</span> <span class="mf">0.5</span><span class="p">,</span> <span class="mf">0.75</span><span class="p">]</span>
<span class="c1"># Identify the cases</span>
<span class="n">is_all_string_type</span> <span class="o">=</span> <span class="p">(</span>
<span class="nb">len</span><span class="p">(</span><span class="n">psser_numeric</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">psser_timestamp</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">psser_string</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span>
<span class="p">)</span>
<span class="n">is_all_numeric_type</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">psser_numeric</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span> <span class="ow">and</span> <span class="nb">len</span><span class="p">(</span><span class="n">psser_timestamp</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span>
<span class="n">has_timestamp_type</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">psser_timestamp</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span>
<span class="n">has_numeric_type</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">psser_numeric</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">0</span>
<span class="k">if</span> <span class="n">is_all_string_type</span><span class="p">:</span>
<span class="c1"># Handling string type columns</span>
<span class="c1"># We will retrive the `count`, `unique`, `top` and `freq`.</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="n">exprs_string</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">_column_label</span><span class="p">)</span> <span class="k">for</span> <span class="n">psser</span> <span class="ow">in</span> <span class="n">psser_string</span>
<span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">exprs_string</span><span class="p">)</span>
<span class="c1"># Get `count` &amp; `unique` for each columns</span>
<span class="n">counts</span><span class="p">,</span> <span class="n">uniques</span> <span class="o">=</span> <span class="nb">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">[</span><span class="mi">1</span><span class="p">:],</span> <span class="n">sdf</span><span class="o">.</span><span class="n">summary</span><span class="p">(</span><span class="s2">&quot;count&quot;</span><span class="p">,</span> <span class="s2">&quot;count_distinct&quot;</span><span class="p">)</span><span class="o">.</span><span class="n">take</span><span class="p">(</span><span class="mi">2</span><span class="p">))</span>
<span class="c1"># Handling Empty DataFrame</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">counts</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span> <span class="ow">or</span> <span class="n">counts</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">==</span> <span class="s2">&quot;0&quot;</span><span class="p">:</span>
<span class="n">data</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">()</span>
<span class="k">for</span> <span class="n">psser</span> <span class="ow">in</span> <span class="n">psser_string</span><span class="p">:</span>
<span class="n">data</span><span class="p">[</span><span class="n">psser</span><span class="o">.</span><span class="n">name</span><span class="p">]</span> <span class="o">=</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="n">np</span><span class="o">.</span><span class="n">nan</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">nan</span><span class="p">]</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">data</span><span class="p">,</span> <span class="n">index</span><span class="o">=</span><span class="p">[</span><span class="s2">&quot;count&quot;</span><span class="p">,</span> <span class="s2">&quot;unique&quot;</span><span class="p">,</span> <span class="s2">&quot;top&quot;</span><span class="p">,</span> <span class="s2">&quot;freq&quot;</span><span class="p">])</span>
<span class="c1"># Get `top` &amp; `freq` for each columns</span>
<span class="n">tops</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">freqs</span> <span class="o">=</span> <span class="p">[]</span>
<span class="c1"># TODO(SPARK-37711): We should do it in single pass since invoking Spark job</span>
<span class="c1"># for every columns is too expensive.</span>
<span class="k">for</span> <span class="n">column</span> <span class="ow">in</span> <span class="n">exprs_string</span><span class="p">:</span>
<span class="n">top</span><span class="p">,</span> <span class="n">freq</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">column</span><span class="p">)</span><span class="o">.</span><span class="n">count</span><span class="p">()</span><span class="o">.</span><span class="n">sort</span><span class="p">(</span><span class="s2">&quot;count&quot;</span><span class="p">,</span> <span class="n">ascending</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span><span class="o">.</span><span class="n">first</span><span class="p">()</span>
<span class="n">tops</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">top</span><span class="p">))</span>
<span class="n">freqs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="nb">str</span><span class="p">(</span><span class="n">freq</span><span class="p">))</span>
<span class="n">stats</span> <span class="o">=</span> <span class="p">[</span><span class="n">counts</span><span class="p">,</span> <span class="n">uniques</span><span class="p">,</span> <span class="n">tops</span><span class="p">,</span> <span class="n">freqs</span><span class="p">]</span>
<span class="n">stats_names</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;count&quot;</span><span class="p">,</span> <span class="s2">&quot;unique&quot;</span><span class="p">,</span> <span class="s2">&quot;top&quot;</span><span class="p">,</span> <span class="s2">&quot;freq&quot;</span><span class="p">]</span>
<span class="n">result</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="n">data</span><span class="o">=</span><span class="n">stats</span><span class="p">,</span>
<span class="n">index</span><span class="o">=</span><span class="n">stats_names</span><span class="p">,</span>
<span class="n">columns</span><span class="o">=</span><span class="n">column_names</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">elif</span> <span class="n">is_all_numeric_type</span><span class="p">:</span>
<span class="c1"># Handling numeric columns</span>
<span class="n">exprs_numeric</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">psser</span><span class="o">.</span><span class="n">_dtype_op</span><span class="o">.</span><span class="n">nan_to_null</span><span class="p">(</span><span class="n">psser</span><span class="p">)</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span> <span class="k">for</span> <span class="n">psser</span> <span class="ow">in</span> <span class="n">psser_numeric</span>
<span class="p">]</span>
<span class="n">formatted_perc</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;</span><span class="si">{:.0%}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">p</span><span class="p">)</span> <span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="nb">sorted</span><span class="p">(</span><span class="n">percentiles</span><span class="p">)]</span>
<span class="n">stats</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;count&quot;</span><span class="p">,</span> <span class="s2">&quot;mean&quot;</span><span class="p">,</span> <span class="s2">&quot;stddev&quot;</span><span class="p">,</span> <span class="s2">&quot;min&quot;</span><span class="p">,</span> <span class="o">*</span><span class="n">formatted_perc</span><span class="p">,</span> <span class="s2">&quot;max&quot;</span><span class="p">]</span>
<span class="c1"># In this case, we can simply use `summary` to calculate the stats.</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">exprs_numeric</span><span class="p">)</span><span class="o">.</span><span class="n">summary</span><span class="p">(</span><span class="o">*</span><span class="n">stats</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span><span class="s2">&quot;stddev&quot;</span><span class="p">,</span> <span class="s2">&quot;std&quot;</span><span class="p">,</span> <span class="n">subset</span><span class="o">=</span><span class="p">[</span><span class="s2">&quot;summary&quot;</span><span class="p">])</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;summary&quot;</span><span class="p">)],</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span>
<span class="p">],</span>
<span class="p">)</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span><span class="s2">&quot;float64&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">has_timestamp_type</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span>
<span class="n">column_names</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">internal</span><span class="o">.</span><span class="n">spark_column_name_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">)</span> <span class="k">for</span> <span class="n">column_label</span> <span class="ow">in</span> <span class="n">column_labels</span>
<span class="p">]</span>
<span class="n">column_length</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span>
<span class="c1"># Apply stat functions for each column.</span>
<span class="n">count_exprs</span> <span class="o">=</span> <span class="nb">map</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">count</span><span class="p">,</span> <span class="n">column_names</span><span class="p">)</span>
<span class="n">min_exprs</span> <span class="o">=</span> <span class="nb">map</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">min</span><span class="p">,</span> <span class="n">column_names</span><span class="p">)</span>
<span class="c1"># Here we try to flat the multiple maps into single list that contains each calculated</span>
<span class="c1"># percentile using `chain`.</span>
<span class="c1"># e.g. flat the `[&lt;map object at 0x7fc1907dc280&gt;, &lt;map object at 0x7fc1907dcc70&gt;]`</span>
<span class="c1"># to `[Column&lt;&#39;percentile_approx(A, 0.2, 10000)&#39;&gt;,</span>
<span class="c1"># Column&lt;&#39;percentile_approx(B, 0.2, 10000)&#39;&gt;,</span>
<span class="c1"># Column&lt;&#39;percentile_approx(A, 0.5, 10000)&#39;&gt;,</span>
<span class="c1"># Column&lt;&#39;percentile_approx(B, 0.5, 10000)&#39;&gt;]`</span>
<span class="n">perc_exprs</span> <span class="o">=</span> <span class="n">chain</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="nb">map</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">percentile_approx</span><span class="p">,</span> <span class="n">column_names</span><span class="p">,</span> <span class="p">[</span><span class="n">percentile</span><span class="p">]</span> <span class="o">*</span> <span class="n">column_length</span><span class="p">)</span>
<span class="k">for</span> <span class="n">percentile</span> <span class="ow">in</span> <span class="n">percentiles</span>
<span class="p">]</span>
<span class="p">)</span>
<span class="n">max_exprs</span> <span class="o">=</span> <span class="nb">map</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">max</span><span class="p">,</span> <span class="n">column_names</span><span class="p">)</span>
<span class="n">mean_exprs</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">column_name</span><span class="p">,</span> <span class="n">spark_data_type</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">column_names</span><span class="p">,</span> <span class="n">spark_data_types</span><span class="p">):</span>
<span class="n">mean_exprs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">mean</span><span class="p">(</span><span class="n">column_name</span><span class="p">)</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span><span class="n">spark_data_type</span><span class="p">))</span>
<span class="n">exprs</span> <span class="o">=</span> <span class="p">[</span><span class="o">*</span><span class="n">count_exprs</span><span class="p">,</span> <span class="o">*</span><span class="n">mean_exprs</span><span class="p">,</span> <span class="o">*</span><span class="n">min_exprs</span><span class="p">,</span> <span class="o">*</span><span class="n">perc_exprs</span><span class="p">,</span> <span class="o">*</span><span class="n">max_exprs</span><span class="p">]</span>
<span class="n">formatted_perc</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;</span><span class="si">{:.0%}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">p</span><span class="p">)</span> <span class="k">for</span> <span class="n">p</span> <span class="ow">in</span> <span class="nb">sorted</span><span class="p">(</span><span class="n">percentiles</span><span class="p">)]</span>
<span class="n">stats_names</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;count&quot;</span><span class="p">,</span> <span class="s2">&quot;mean&quot;</span><span class="p">,</span> <span class="s2">&quot;min&quot;</span><span class="p">,</span> <span class="o">*</span><span class="n">formatted_perc</span><span class="p">,</span> <span class="s2">&quot;max&quot;</span><span class="p">]</span>
<span class="c1"># If not all columns are timestamp type,</span>
<span class="c1"># we also need to calculate the `std` for numeric columns</span>
<span class="k">if</span> <span class="n">has_numeric_type</span><span class="p">:</span>
<span class="n">std_exprs</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">spark_data_type</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">spark_data_types</span><span class="p">):</span>
<span class="n">column_name</span> <span class="o">=</span> <span class="n">label</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">spark_data_type</span><span class="p">,</span> <span class="p">(</span><span class="n">TimestampType</span><span class="p">,</span> <span class="n">TimestampNTZType</span><span class="p">)):</span>
<span class="n">std_exprs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="s2">&quot;stddev_samp(</span><span class="si">{}</span><span class="s2">)&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">column_name</span><span class="p">)))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">std_exprs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">stddev</span><span class="p">(</span><span class="n">column_name</span><span class="p">))</span>
<span class="n">exprs</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">std_exprs</span><span class="p">)</span>
<span class="n">stats_names</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="s2">&quot;std&quot;</span><span class="p">)</span>
<span class="c1"># Select stats for all columns at once.</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">exprs</span><span class="p">)</span>
<span class="n">stat_values</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">first</span><span class="p">()</span>
<span class="n">num_stats</span> <span class="o">=</span> <span class="nb">int</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">exprs</span><span class="p">)</span> <span class="o">/</span> <span class="n">column_length</span><span class="p">)</span>
<span class="c1"># `column_name_stats_kv` is key-value store that has column name as key, and</span>
<span class="c1"># the stats as values e.g. {&quot;A&quot;: [{count_value}, {min_value}, ...],</span>
<span class="c1"># &quot;B&quot;: [{count_value}, {min_value} ...]}</span>
<span class="n">column_name_stats_kv</span><span class="p">:</span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]</span> <span class="o">=</span> <span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">)</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">column_name</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">column_names</span><span class="p">):</span>
<span class="k">for</span> <span class="n">first_stat_idx</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">num_stats</span><span class="p">):</span>
<span class="n">column_name_stats_kv</span><span class="p">[</span><span class="n">column_name</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">stat_values</span><span class="p">[(</span><span class="n">first_stat_idx</span> <span class="o">*</span> <span class="n">column_length</span><span class="p">)</span> <span class="o">+</span> <span class="n">i</span><span class="p">]</span>
<span class="p">)</span>
<span class="c1"># For timestamp type columns, we should cast the column type to string.</span>
<span class="k">for</span> <span class="n">key</span><span class="p">,</span> <span class="n">spark_data_type</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">column_name_stats_kv</span><span class="p">,</span> <span class="n">spark_data_types</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">spark_data_type</span><span class="p">,</span> <span class="p">(</span><span class="n">TimestampType</span><span class="p">,</span> <span class="n">TimestampNTZType</span><span class="p">)):</span>
<span class="n">column_name_stats_kv</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</span> <span class="p">[</span><span class="nb">str</span><span class="p">(</span><span class="n">value</span><span class="p">)</span> <span class="k">for</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">column_name_stats_kv</span><span class="p">[</span><span class="n">key</span><span class="p">]]</span>
<span class="n">result</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span> <span class="c1"># type: ignore[no-redef]</span>
<span class="n">data</span><span class="o">=</span><span class="n">column_name_stats_kv</span><span class="p">,</span>
<span class="n">index</span><span class="o">=</span><span class="n">stats_names</span><span class="p">,</span>
<span class="n">columns</span><span class="o">=</span><span class="n">column_names</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Empty DataFrame without column</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Cannot describe a DataFrame without columns&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="DataFrame.drop_duplicates"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.drop_duplicates.html#pyspark.pandas.DataFrame.drop_duplicates">[docs]</a> <span class="k">def</span> <span class="nf">drop_duplicates</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">subset</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">keep</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">bool</span><span class="p">,</span> <span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="s2">&quot;first&quot;</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">ignore_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return DataFrame with duplicate rows removed, optionally only</span>
<span class="sd"> considering certain columns.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> subset : column label or sequence of labels, optional</span>
<span class="sd"> Only consider certain columns for identifying duplicates, by</span>
<span class="sd"> default use all the columns.</span>
<span class="sd"> keep : {&#39;first&#39;, &#39;last&#39;, False}, default &#39;first&#39;</span>
<span class="sd"> Determines which duplicates (if any) to keep.</span>
<span class="sd"> - ``first`` : Drop duplicates except for the first occurrence.</span>
<span class="sd"> - ``last`` : Drop duplicates except for the last occurrence.</span>
<span class="sd"> - False : Drop all duplicates.</span>
<span class="sd"> inplace : boolean, default False</span>
<span class="sd"> Whether to drop duplicates in place or to return a copy.</span>
<span class="sd"> ignore_index : boolean, default False</span>
<span class="sd"> If True, the resulting axis will be labeled 0, 1, …, n - 1.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame with duplicates removed or None if ``inplace=True``.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(</span>
<span class="sd"> ... {&#39;a&#39;: [1, 2, 2, 2, 3], &#39;b&#39;: [&#39;a&#39;, &#39;a&#39;, &#39;a&#39;, &#39;c&#39;, &#39;d&#39;]}, columns = [&#39;a&#39;, &#39;b&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 a</span>
<span class="sd"> 1 2 a</span>
<span class="sd"> 2 2 a</span>
<span class="sd"> 3 2 c</span>
<span class="sd"> 4 3 d</span>
<span class="sd"> &gt;&gt;&gt; df.drop_duplicates().sort_index()</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 a</span>
<span class="sd"> 1 2 a</span>
<span class="sd"> 3 2 c</span>
<span class="sd"> 4 3 d</span>
<span class="sd"> &gt;&gt;&gt; df.drop_duplicates(ignore_index=True).sort_index()</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 a</span>
<span class="sd"> 1 2 a</span>
<span class="sd"> 2 2 c</span>
<span class="sd"> 3 3 d</span>
<span class="sd"> &gt;&gt;&gt; df.drop_duplicates(&#39;a&#39;).sort_index()</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 a</span>
<span class="sd"> 1 2 a</span>
<span class="sd"> 4 3 d</span>
<span class="sd"> &gt;&gt;&gt; df.drop_duplicates([&#39;a&#39;, &#39;b&#39;]).sort_index()</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 a</span>
<span class="sd"> 1 2 a</span>
<span class="sd"> 3 2 c</span>
<span class="sd"> 4 3 d</span>
<span class="sd"> &gt;&gt;&gt; df.drop_duplicates(keep=&#39;last&#39;).sort_index()</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 a</span>
<span class="sd"> 2 2 a</span>
<span class="sd"> 3 2 c</span>
<span class="sd"> 4 3 d</span>
<span class="sd"> &gt;&gt;&gt; df.drop_duplicates(keep=False).sort_index()</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 a</span>
<span class="sd"> 3 2 c</span>
<span class="sd"> 4 3 d</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="n">sdf</span><span class="p">,</span> <span class="n">column</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_mark_duplicates</span><span class="p">(</span><span class="n">subset</span><span class="p">,</span> <span class="n">keep</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">where</span><span class="p">(</span><span class="o">~</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">column</span><span class="p">))</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">column</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">sdf</span><span class="p">)</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="k">if</span> <span class="n">ignore_index</span><span class="p">:</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">inplace</span><span class="o">=</span><span class="n">inplace</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> <span class="k">if</span> <span class="n">ignore_index</span> <span class="k">else</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.reindex"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.reindex.html#pyspark.pandas.DataFrame.reindex">[docs]</a> <span class="k">def</span> <span class="nf">reindex</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">labels</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="s2">&quot;Index&quot;</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Axis</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">copy</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">fill_value</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Any</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Conform DataFrame to new index with optional filling logic, placing</span>
<span class="sd"> NA/NaN in locations having no value in the previous index. A new object</span>
<span class="sd"> is produced unless the new index is equivalent to the current one and</span>
<span class="sd"> ``copy=False``.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> labels: array-like, optional</span>
<span class="sd"> New labels / index to conform the axis specified by ‘axis’ to.</span>
<span class="sd"> index, columns: array-like, optional</span>
<span class="sd"> New labels / index to conform to, should be specified using keywords.</span>
<span class="sd"> Preferably an Index object to avoid duplicating data</span>
<span class="sd"> axis: int or str, optional</span>
<span class="sd"> Axis to target. Can be either the axis name (‘index’, ‘columns’) or</span>
<span class="sd"> number (0, 1).</span>
<span class="sd"> copy : bool, default True</span>
<span class="sd"> Return a new object, even if the passed indexes are the same.</span>
<span class="sd"> fill_value : scalar, default np.NaN</span>
<span class="sd"> Value to use for missing values. Defaults to NaN, but can be any</span>
<span class="sd"> &quot;compatible&quot; value.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame with changed index.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.set_index : Set row labels.</span>
<span class="sd"> DataFrame.reset_index : Remove row labels or move them to new columns.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> ``DataFrame.reindex`` supports two calling conventions</span>
<span class="sd"> * ``(index=index_labels, columns=column_labels, ...)``</span>
<span class="sd"> * ``(labels, axis={&#39;index&#39;, &#39;columns&#39;}, ...)``</span>
<span class="sd"> We *highly* recommend using keyword arguments to clarify your</span>
<span class="sd"> intent.</span>
<span class="sd"> Create a dataframe with some fictional data.</span>
<span class="sd"> &gt;&gt;&gt; index = [&#39;Firefox&#39;, &#39;Chrome&#39;, &#39;Safari&#39;, &#39;IE10&#39;, &#39;Konqueror&#39;]</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({</span>
<span class="sd"> ... &#39;http_status&#39;: [200, 200, 404, 404, 301],</span>
<span class="sd"> ... &#39;response_time&#39;: [0.04, 0.02, 0.07, 0.08, 1.0]},</span>
<span class="sd"> ... index=index,</span>
<span class="sd"> ... columns=[&#39;http_status&#39;, &#39;response_time&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> http_status response_time</span>
<span class="sd"> Firefox 200 0.04</span>
<span class="sd"> Chrome 200 0.02</span>
<span class="sd"> Safari 404 0.07</span>
<span class="sd"> IE10 404 0.08</span>
<span class="sd"> Konqueror 301 1.00</span>
<span class="sd"> Create a new index and reindex the dataframe. By default</span>
<span class="sd"> values in the new index that do not have corresponding</span>
<span class="sd"> records in the dataframe are assigned ``NaN``.</span>
<span class="sd"> &gt;&gt;&gt; new_index= [&#39;Safari&#39;, &#39;Iceweasel&#39;, &#39;Comodo Dragon&#39;, &#39;IE10&#39;,</span>
<span class="sd"> ... &#39;Chrome&#39;]</span>
<span class="sd"> &gt;&gt;&gt; df.reindex(new_index).sort_index()</span>
<span class="sd"> http_status response_time</span>
<span class="sd"> Chrome 200.0 0.02</span>
<span class="sd"> Comodo Dragon NaN NaN</span>
<span class="sd"> IE10 404.0 0.08</span>
<span class="sd"> Iceweasel NaN NaN</span>
<span class="sd"> Safari 404.0 0.07</span>
<span class="sd"> We can fill in the missing values by passing a value to</span>
<span class="sd"> the keyword ``fill_value``.</span>
<span class="sd"> &gt;&gt;&gt; df.reindex(new_index, fill_value=0, copy=False).sort_index()</span>
<span class="sd"> http_status response_time</span>
<span class="sd"> Chrome 200 0.02</span>
<span class="sd"> Comodo Dragon 0 0.00</span>
<span class="sd"> IE10 404 0.08</span>
<span class="sd"> Iceweasel 0 0.00</span>
<span class="sd"> Safari 404 0.07</span>
<span class="sd"> We can also reindex the columns.</span>
<span class="sd"> &gt;&gt;&gt; df.reindex(columns=[&#39;http_status&#39;, &#39;user_agent&#39;]).sort_index()</span>
<span class="sd"> http_status user_agent</span>
<span class="sd"> Chrome 200 NaN</span>
<span class="sd"> Firefox 200 NaN</span>
<span class="sd"> IE10 404 NaN</span>
<span class="sd"> Konqueror 301 NaN</span>
<span class="sd"> Safari 404 NaN</span>
<span class="sd"> Or we can use &quot;axis-style&quot; keyword arguments</span>
<span class="sd"> &gt;&gt;&gt; df.reindex([&#39;http_status&#39;, &#39;user_agent&#39;], axis=&quot;columns&quot;).sort_index()</span>
<span class="sd"> http_status user_agent</span>
<span class="sd"> Chrome 200 NaN</span>
<span class="sd"> Firefox 200 NaN</span>
<span class="sd"> IE10 404 NaN</span>
<span class="sd"> Konqueror 301 NaN</span>
<span class="sd"> Safari 404 NaN</span>
<span class="sd"> To further illustrate the filling functionality in</span>
<span class="sd"> ``reindex``, we will create a dataframe with a</span>
<span class="sd"> monotonically increasing index (for example, a sequence</span>
<span class="sd"> of dates).</span>
<span class="sd"> &gt;&gt;&gt; date_index = pd.date_range(&#39;1/1/2010&#39;, periods=6, freq=&#39;D&#39;)</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame({&quot;prices&quot;: [100, 101, np.nan, 100, 89, 88]},</span>
<span class="sd"> ... index=date_index)</span>
<span class="sd"> &gt;&gt;&gt; df2.sort_index()</span>
<span class="sd"> prices</span>
<span class="sd"> 2010-01-01 100.0</span>
<span class="sd"> 2010-01-02 101.0</span>
<span class="sd"> 2010-01-03 NaN</span>
<span class="sd"> 2010-01-04 100.0</span>
<span class="sd"> 2010-01-05 89.0</span>
<span class="sd"> 2010-01-06 88.0</span>
<span class="sd"> Suppose we decide to expand the dataframe to cover a wider</span>
<span class="sd"> date range.</span>
<span class="sd"> &gt;&gt;&gt; date_index2 = pd.date_range(&#39;12/29/2009&#39;, periods=10, freq=&#39;D&#39;)</span>
<span class="sd"> &gt;&gt;&gt; df2.reindex(date_index2).sort_index()</span>
<span class="sd"> prices</span>
<span class="sd"> 2009-12-29 NaN</span>
<span class="sd"> 2009-12-30 NaN</span>
<span class="sd"> 2009-12-31 NaN</span>
<span class="sd"> 2010-01-01 100.0</span>
<span class="sd"> 2010-01-02 101.0</span>
<span class="sd"> 2010-01-03 NaN</span>
<span class="sd"> 2010-01-04 100.0</span>
<span class="sd"> 2010-01-05 89.0</span>
<span class="sd"> 2010-01-06 88.0</span>
<span class="sd"> 2010-01-07 NaN</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">axis</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="p">(</span><span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Cannot specify both &#39;axis&#39; and any of &#39;index&#39; or &#39;columns&#39;.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">labels</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">index</span> <span class="o">=</span> <span class="n">labels</span>
<span class="k">elif</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">columns</span> <span class="o">=</span> <span class="n">labels</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">index</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Index must be called with a collection of some kind, &quot;</span>
<span class="s2">&quot;</span><span class="si">%s</span><span class="s2"> was passed&quot;</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">index</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">columns</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Columns must be called with a collection of some kind, &quot;</span>
<span class="s2">&quot;</span><span class="si">%s</span><span class="s2"> was passed&quot;</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">df</span> <span class="o">=</span> <span class="bp">self</span>
<span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">_reindex_index</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="n">fill_value</span><span class="p">)</span>
<span class="k">if</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">_reindex_columns</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="n">fill_value</span><span class="p">)</span>
<span class="c1"># Copy</span>
<span class="k">if</span> <span class="n">copy</span> <span class="ow">and</span> <span class="n">df</span> <span class="ow">is</span> <span class="bp">self</span><span class="p">:</span>
<span class="k">return</span> <span class="n">df</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">df</span></div>
<span class="k">def</span> <span class="nf">_reindex_index</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">index</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="s2">&quot;Index&quot;</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">]]],</span> <span class="n">fill_value</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Any</span><span class="p">]</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="c1"># When axis is index, we can mimic pandas by a right outer join.</span>
<span class="n">nlevels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span>
<span class="k">assert</span> <span class="n">nlevels</span> <span class="o">&lt;=</span> <span class="mi">1</span> <span class="ow">or</span> <span class="p">(</span>
<span class="nb">isinstance</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">)</span> <span class="ow">and</span> <span class="n">nlevels</span> <span class="o">==</span> <span class="n">index</span><span class="o">.</span><span class="n">nlevels</span>
<span class="p">),</span> <span class="s2">&quot;MultiIndex DataFrame can only be reindexed with a similar pandas-on-Spark MultiIndex.&quot;</span>
<span class="n">index_columns</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">frame</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">NATURAL_ORDER_COLUMN_NAME</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">Index</span><span class="p">):</span>
<span class="k">if</span> <span class="n">nlevels</span> <span class="o">!=</span> <span class="n">index</span><span class="o">.</span><span class="n">nlevels</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">index</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">([]))</span><span class="o">.</span><span class="n">reindex</span><span class="p">(</span>
<span class="n">columns</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">,</span> <span class="n">fill_value</span><span class="o">=</span><span class="n">fill_value</span>
<span class="p">)</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">index</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span>
<span class="n">scols</span> <span class="o">=</span> <span class="n">index</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="n">labels</span> <span class="o">=</span> <span class="n">index</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="p">[</span><span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_column</span><span class="p">)</span> <span class="k">for</span> <span class="n">scol</span><span class="p">,</span> <span class="n">index_column</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">scols</span><span class="p">,</span> <span class="n">index_columns</span><span class="p">)]</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">index</span> <span class="o">=</span> <span class="n">ps</span><span class="o">.</span><span class="n">Index</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="n">index</span><span class="p">))</span>
<span class="n">labels</span> <span class="o">=</span> <span class="n">index</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">index</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_columns</span><span class="p">[</span><span class="mi">0</span><span class="p">]))</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span>
<span class="k">if</span> <span class="n">fill_value</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">frame_index_columns</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">frame</span><span class="p">,</span> <span class="s2">&quot;__frame_index_column_</span><span class="si">{}</span><span class="s2">__&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">i</span><span class="p">))</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">nlevels</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">index_scols</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">frame</span><span class="p">,</span> <span class="n">index_col</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">frame_index_col</span><span class="p">)</span>
<span class="k">for</span> <span class="n">index_col</span><span class="p">,</span> <span class="n">frame_index_col</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">index_columns</span><span class="p">,</span> <span class="n">frame_index_columns</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">scols</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">data_spark_columns</span>
<span class="n">frame</span> <span class="o">=</span> <span class="n">frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">index_scols</span> <span class="o">+</span> <span class="n">scols</span><span class="p">)</span>
<span class="n">temp_fill_value</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">frame</span><span class="p">,</span> <span class="s2">&quot;__fill_value__&quot;</span><span class="p">)</span>
<span class="n">labels</span> <span class="o">=</span> <span class="n">labels</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="n">temp_fill_value</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">fill_value</span><span class="p">))</span>
<span class="n">frame_index_scols</span> <span class="o">=</span> <span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">frame</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">frame_index_columns</span><span class="p">]</span>
<span class="n">labels_index_scols</span> <span class="o">=</span> <span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">labels</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">index_columns</span><span class="p">]</span>
<span class="n">joined_df</span> <span class="o">=</span> <span class="n">frame</span><span class="o">.</span><span class="n">join</span><span class="p">(</span>
<span class="n">labels</span><span class="p">,</span>
<span class="n">on</span><span class="o">=</span><span class="p">[</span><span class="n">fcol</span> <span class="o">==</span> <span class="n">lcol</span> <span class="k">for</span> <span class="n">fcol</span><span class="p">,</span> <span class="n">lcol</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">frame_index_scols</span><span class="p">,</span> <span class="n">labels_index_scols</span><span class="p">)],</span>
<span class="n">how</span><span class="o">=</span><span class="s2">&quot;right&quot;</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">joined_df</span> <span class="o">=</span> <span class="n">joined_df</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="o">*</span><span class="n">labels_index_scols</span><span class="p">,</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span>
<span class="n">reduce</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">c1</span><span class="p">,</span> <span class="n">c2</span><span class="p">:</span> <span class="n">c1</span> <span class="o">&amp;</span> <span class="n">c2</span><span class="p">,</span>
<span class="p">[</span>
<span class="n">fcol</span><span class="o">.</span><span class="n">isNull</span><span class="p">()</span> <span class="o">&amp;</span> <span class="n">lcol</span><span class="o">.</span><span class="n">isNotNull</span><span class="p">()</span>
<span class="k">for</span> <span class="n">fcol</span><span class="p">,</span> <span class="n">lcol</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">frame_index_scols</span><span class="p">,</span> <span class="n">labels_index_scols</span><span class="p">)</span>
<span class="p">],</span>
<span class="p">),</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">joined_df</span><span class="p">,</span> <span class="n">temp_fill_value</span><span class="p">),</span>
<span class="p">)</span>
<span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">scol_for</span><span class="p">(</span><span class="n">joined_df</span><span class="p">,</span> <span class="n">col</span><span class="p">))</span>
<span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="p">],</span>
<span class="p">)</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">joined_df</span> <span class="o">=</span> <span class="n">frame</span><span class="o">.</span><span class="n">join</span><span class="p">(</span><span class="n">labels</span><span class="p">,</span> <span class="n">on</span><span class="o">=</span><span class="n">index_columns</span><span class="p">,</span> <span class="n">how</span><span class="o">=</span><span class="s2">&quot;right&quot;</span><span class="p">)</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="p">[</span><span class="n">field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">nullable</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> <span class="k">for</span> <span class="n">field</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">joined_df</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">NATURAL_ORDER_COLUMN_NAME</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index_names</span><span class="p">,</span>
<span class="n">index_fields</span><span class="o">=</span><span class="p">[</span>
<span class="n">field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">name</span><span class="p">)</span>
<span class="k">for</span> <span class="n">field</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span>
<span class="n">index</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="p">)</span>
<span class="p">],</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="p">],</span>
<span class="n">data_fields</span><span class="o">=</span><span class="n">data_fields</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">internal</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_reindex_columns</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">]]],</span> <span class="n">fill_value</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Any</span><span class="p">]</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="n">level</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span>
<span class="k">if</span> <span class="n">level</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">label_columns</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span>
<span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">label_columns</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">col</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Expected tuple, got </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">col</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">label_columns</span> <span class="o">=</span> <span class="p">[(</span><span class="n">col</span><span class="p">,)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">columns</span><span class="p">]</span>
<span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">label_columns</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">col</span><span class="p">)</span> <span class="o">!=</span> <span class="n">level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;shape (1,</span><span class="si">{}</span><span class="s2">) doesn&#39;t match the shape (1,</span><span class="si">{}</span><span class="s2">)&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">col</span><span class="p">),</span> <span class="n">level</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">fill_value</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">nan</span> <span class="k">if</span> <span class="n">fill_value</span> <span class="ow">is</span> <span class="kc">None</span> <span class="k">else</span> <span class="n">fill_value</span>
<span class="n">scols_or_pssers</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">,</span> <span class="s2">&quot;Series&quot;</span><span class="p">]]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">labels</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">label_columns</span><span class="p">:</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">scols_or_pssers</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">scols_or_pssers</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">fill_value</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">)))</span>
<span class="n">labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">):</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">name</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="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">columns</span><span class="o">.</span><span class="n">names</span>
<span class="p">]</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span>
<span class="n">scols_or_pssers</span><span class="p">,</span> <span class="n">column_labels</span><span class="o">=</span><span class="n">labels</span><span class="p">,</span> <span class="n">column_label_names</span><span class="o">=</span><span class="n">column_label_names</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">scols_or_pssers</span><span class="p">,</span> <span class="n">column_labels</span><span class="o">=</span><span class="n">labels</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.reindex_like"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.reindex_like.html#pyspark.pandas.DataFrame.reindex_like">[docs]</a> <span class="k">def</span> <span class="nf">reindex_like</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="s2">&quot;DataFrame&quot;</span><span class="p">,</span> <span class="n">copy</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a DataFrame with matching indices as other object.</span>
<span class="sd"> Conform the object to the same index on all axes. Places NA/NaN in locations</span>
<span class="sd"> having no value in the previous index. A new object is produced unless the</span>
<span class="sd"> new index is equivalent to the current one and copy=False.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> other : DataFrame</span>
<span class="sd"> Its row and column indices are used to define the new indices</span>
<span class="sd"> of this object.</span>
<span class="sd"> copy : bool, default True</span>
<span class="sd"> Return a new object, even if the passed indexes are the same.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame with changed indices on each axis.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.set_index : Set row labels.</span>
<span class="sd"> DataFrame.reset_index : Remove row labels or move them to new columns.</span>
<span class="sd"> DataFrame.reindex : Change to new indices or expand indices.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> Same as calling</span>
<span class="sd"> ``.reindex(index=other.index, columns=other.columns,...)``.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame([[24.3, 75.7, &#39;high&#39;],</span>
<span class="sd"> ... [31, 87.8, &#39;high&#39;],</span>
<span class="sd"> ... [22, 71.6, &#39;medium&#39;],</span>
<span class="sd"> ... [35, 95, &#39;medium&#39;]],</span>
<span class="sd"> ... columns=[&#39;temp_celsius&#39;, &#39;temp_fahrenheit&#39;,</span>
<span class="sd"> ... &#39;windspeed&#39;],</span>
<span class="sd"> ... index=pd.date_range(start=&#39;2014-02-12&#39;,</span>
<span class="sd"> ... end=&#39;2014-02-15&#39;, freq=&#39;D&#39;))</span>
<span class="sd"> &gt;&gt;&gt; df1</span>
<span class="sd"> temp_celsius temp_fahrenheit windspeed</span>
<span class="sd"> 2014-02-12 24.3 75.7 high</span>
<span class="sd"> 2014-02-13 31.0 87.8 high</span>
<span class="sd"> 2014-02-14 22.0 71.6 medium</span>
<span class="sd"> 2014-02-15 35.0 95.0 medium</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame([[28, &#39;low&#39;],</span>
<span class="sd"> ... [30, &#39;low&#39;],</span>
<span class="sd"> ... [35.1, &#39;medium&#39;]],</span>
<span class="sd"> ... columns=[&#39;temp_celsius&#39;, &#39;windspeed&#39;],</span>
<span class="sd"> ... index=pd.DatetimeIndex([&#39;2014-02-12&#39;, &#39;2014-02-13&#39;,</span>
<span class="sd"> ... &#39;2014-02-15&#39;]))</span>
<span class="sd"> &gt;&gt;&gt; df2</span>
<span class="sd"> temp_celsius windspeed</span>
<span class="sd"> 2014-02-12 28.0 low</span>
<span class="sd"> 2014-02-13 30.0 low</span>
<span class="sd"> 2014-02-15 35.1 medium</span>
<span class="sd"> &gt;&gt;&gt; df2.reindex_like(df1).sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> temp_celsius temp_fahrenheit windspeed</span>
<span class="sd"> 2014-02-12 28.0 NaN low</span>
<span class="sd"> 2014-02-13 30.0 NaN low</span>
<span class="sd"> 2014-02-14 NaN NaN None</span>
<span class="sd"> 2014-02-15 35.1 NaN medium</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">reindex</span><span class="p">(</span><span class="n">index</span><span class="o">=</span><span class="n">other</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">other</span><span class="o">.</span><span class="n">columns</span><span class="p">,</span> <span class="n">copy</span><span class="o">=</span><span class="n">copy</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;other must be a pandas-on-Spark DataFrame&quot;</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.melt"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.melt.html#pyspark.pandas.DataFrame.melt">[docs]</a> <span class="k">def</span> <span class="nf">melt</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">id_vars</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">value_vars</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">var_name</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">value_name</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;value&quot;</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Unpivot a DataFrame from wide format to long format, optionally</span>
<span class="sd"> leaving identifier variables set.</span>
<span class="sd"> This function is useful to massage a DataFrame into a format where one</span>
<span class="sd"> or more columns are identifier variables (`id_vars`), while all other</span>
<span class="sd"> columns, considered measured variables (`value_vars`), are &quot;unpivoted&quot; to</span>
<span class="sd"> the row axis, leaving just two non-identifier columns, &#39;variable&#39; and</span>
<span class="sd"> &#39;value&#39;.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> frame : DataFrame</span>
<span class="sd"> id_vars : tuple, list, or ndarray, optional</span>
<span class="sd"> Column(s) to use as identifier variables.</span>
<span class="sd"> value_vars : tuple, list, or ndarray, optional</span>
<span class="sd"> Column(s) to unpivot. If not specified, uses all columns that</span>
<span class="sd"> are not set as `id_vars`.</span>
<span class="sd"> var_name : scalar, default &#39;variable&#39;</span>
<span class="sd"> Name to use for the &#39;variable&#39; column. If None it uses `frame.columns.name` or</span>
<span class="sd"> ‘variable’.</span>
<span class="sd"> value_name : scalar, default &#39;value&#39;</span>
<span class="sd"> Name to use for the &#39;value&#39; column.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Unpivoted DataFrame.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: {0: &#39;a&#39;, 1: &#39;b&#39;, 2: &#39;c&#39;},</span>
<span class="sd"> ... &#39;B&#39;: {0: 1, 1: 3, 2: 5},</span>
<span class="sd"> ... &#39;C&#39;: {0: 2, 1: 4, 2: 6}},</span>
<span class="sd"> ... columns=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C</span>
<span class="sd"> 0 a 1 2</span>
<span class="sd"> 1 b 3 4</span>
<span class="sd"> 2 c 5 6</span>
<span class="sd"> &gt;&gt;&gt; ps.melt(df)</span>
<span class="sd"> variable value</span>
<span class="sd"> 0 A a</span>
<span class="sd"> 1 B 1</span>
<span class="sd"> 2 C 2</span>
<span class="sd"> 3 A b</span>
<span class="sd"> 4 B 3</span>
<span class="sd"> 5 C 4</span>
<span class="sd"> 6 A c</span>
<span class="sd"> 7 B 5</span>
<span class="sd"> 8 C 6</span>
<span class="sd"> &gt;&gt;&gt; df.melt(id_vars=&#39;A&#39;)</span>
<span class="sd"> A variable value</span>
<span class="sd"> 0 a B 1</span>
<span class="sd"> 1 a C 2</span>
<span class="sd"> 2 b B 3</span>
<span class="sd"> 3 b C 4</span>
<span class="sd"> 4 c B 5</span>
<span class="sd"> 5 c C 6</span>
<span class="sd"> &gt;&gt;&gt; df.melt(value_vars=&#39;A&#39;)</span>
<span class="sd"> variable value</span>
<span class="sd"> 0 A a</span>
<span class="sd"> 1 A b</span>
<span class="sd"> 2 A c</span>
<span class="sd"> &gt;&gt;&gt; ps.melt(df, id_vars=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> A B variable value</span>
<span class="sd"> 0 a 1 C 2</span>
<span class="sd"> 1 b 3 C 4</span>
<span class="sd"> 2 c 5 C 6</span>
<span class="sd"> &gt;&gt;&gt; df.melt(id_vars=[&#39;A&#39;], value_vars=[&#39;C&#39;])</span>
<span class="sd"> A variable value</span>
<span class="sd"> 0 a C 2</span>
<span class="sd"> 1 b C 4</span>
<span class="sd"> 2 c C 6</span>
<span class="sd"> The names of &#39;variable&#39; and &#39;value&#39; columns can be customized:</span>
<span class="sd"> &gt;&gt;&gt; ps.melt(df, id_vars=[&#39;A&#39;], value_vars=[&#39;B&#39;],</span>
<span class="sd"> ... var_name=&#39;myVarname&#39;, value_name=&#39;myValname&#39;)</span>
<span class="sd"> A myVarname myValname</span>
<span class="sd"> 0 a B 1</span>
<span class="sd"> 1 b B 3</span>
<span class="sd"> 2 c B 5</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">if</span> <span class="n">id_vars</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">id_vars</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">id_vars</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">id_vars</span> <span class="o">=</span> <span class="p">[</span><span class="n">idv</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">idv</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">idv</span><span class="p">,)</span> <span class="k">for</span> <span class="n">idv</span> <span class="ow">in</span> <span class="n">id_vars</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;id_vars must be a list of tuples&quot;</span> <span class="s2">&quot; when columns are a MultiIndex&quot;</span>
<span class="p">)</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">id_vars</span><span class="p">):</span>
<span class="n">id_vars</span> <span class="o">=</span> <span class="p">[(</span><span class="n">id_vars</span><span class="p">,)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">id_vars</span> <span class="o">=</span> <span class="p">[</span><span class="n">idv</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">idv</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">idv</span><span class="p">,)</span> <span class="k">for</span> <span class="n">idv</span> <span class="ow">in</span> <span class="n">id_vars</span><span class="p">]</span>
<span class="n">non_existence_col</span> <span class="o">=</span> <span class="p">[</span><span class="n">idv</span> <span class="k">for</span> <span class="n">idv</span> <span class="ow">in</span> <span class="n">id_vars</span> <span class="k">if</span> <span class="n">idv</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">non_existence_col</span><span class="p">)</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">raveled_column_labels</span><span class="p">:</span> <span class="n">np</span><span class="o">.</span><span class="n">ndarray</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">np</span><span class="o">.</span><span class="n">dtype</span><span class="p">[</span><span class="n">Any</span><span class="p">]]</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">ravel</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span>
<span class="n">missing</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">nec</span> <span class="k">for</span> <span class="n">nec</span> <span class="ow">in</span> <span class="n">np</span><span class="o">.</span><span class="n">ravel</span><span class="p">(</span><span class="n">non_existence_col</span><span class="p">)</span> <span class="k">if</span> <span class="n">nec</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">raveled_column_labels</span>
<span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">missing</span><span class="p">)</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span>
<span class="s2">&quot;The following &#39;id_vars&#39; are not present&quot;</span>
<span class="s2">&quot; in the DataFrame: </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">missing</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span>
<span class="s2">&quot;None of </span><span class="si">{}</span><span class="s2"> are in the </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">non_existence_col</span><span class="p">,</span> <span class="n">column_labels</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">value_vars</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">value_vars</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value_vars</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">value_vars</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">valv</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">valv</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">valv</span><span class="p">,)</span> <span class="k">for</span> <span class="n">valv</span> <span class="ow">in</span> <span class="n">value_vars</span>
<span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;value_vars must be a list of tuples&quot;</span> <span class="s2">&quot; when columns are a MultiIndex&quot;</span>
<span class="p">)</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">value_vars</span><span class="p">):</span>
<span class="n">value_vars</span> <span class="o">=</span> <span class="p">[(</span><span class="n">value_vars</span><span class="p">,)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">value_vars</span> <span class="o">=</span> <span class="p">[</span><span class="n">valv</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">valv</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">valv</span><span class="p">,)</span> <span class="k">for</span> <span class="n">valv</span> <span class="ow">in</span> <span class="n">value_vars</span><span class="p">]</span>
<span class="n">non_existence_col</span> <span class="o">=</span> <span class="p">[</span><span class="n">valv</span> <span class="k">for</span> <span class="n">valv</span> <span class="ow">in</span> <span class="n">value_vars</span> <span class="k">if</span> <span class="n">valv</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">non_existence_col</span><span class="p">)</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">raveled_column_labels</span> <span class="o">=</span> <span class="n">np</span><span class="o">.</span><span class="n">ravel</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span>
<span class="n">missing</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">nec</span> <span class="k">for</span> <span class="n">nec</span> <span class="ow">in</span> <span class="n">np</span><span class="o">.</span><span class="n">ravel</span><span class="p">(</span><span class="n">non_existence_col</span><span class="p">)</span> <span class="k">if</span> <span class="n">nec</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">raveled_column_labels</span>
<span class="p">]</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">missing</span><span class="p">)</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span>
<span class="s2">&quot;The following &#39;value_vars&#39; are not present&quot;</span>
<span class="s2">&quot; in the DataFrame: </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">missing</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span>
<span class="s2">&quot;None of </span><span class="si">{}</span><span class="s2"> are in the </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">non_existence_col</span><span class="p">,</span> <span class="n">column_labels</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">value_vars</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">value_vars</span> <span class="o">=</span> <span class="n">column_labels</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">label</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span> <span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">id_vars</span><span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="k">if</span> <span class="n">var_name</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">==</span> <span class="mi">1</span>
<span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="p">):</span>
<span class="n">var_name</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;variable&quot;</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">var_name</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">name_like_string</span><span class="p">(</span><span class="n">name</span><span class="p">)</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="k">else</span> <span class="s2">&quot;variable_</span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">)</span>
<span class="p">]</span>
<span class="k">elif</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">var_name</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="sa">f</span><span class="s2">&quot;</span><span class="si">{</span><span class="n">var_name</span><span class="si">=}</span><span class="s2"> must be a scalar.&quot;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">var_name</span> <span class="o">=</span> <span class="p">[</span><span class="n">var_name</span><span class="p">]</span> <span class="c1"># type: ignore[list-item]</span>
<span class="n">pairs</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">c</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">label</span><span class="p">,</span> <span class="n">var_name</span><span class="p">)],</span>
<span class="o">*</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">value_name</span><span class="p">)],</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">value_vars</span>
<span class="p">]</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">columns</span> <span class="o">=</span> <span class="p">(</span>
<span class="p">[</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">id_vars</span>
<span class="p">]</span>
<span class="o">+</span> <span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;pairs.`</span><span class="si">%s</span><span class="s2">`&quot;</span> <span class="o">%</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">var_name</span><span class="p">]</span>
<span class="o">+</span> <span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;pairs.`</span><span class="si">%s</span><span class="s2">`&quot;</span> <span class="o">%</span> <span class="n">value_name</span><span class="p">)]</span>
<span class="p">)</span>
<span class="n">exploded_df</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="s2">&quot;pairs&quot;</span><span class="p">,</span> <span class="n">pairs</span><span class="p">)</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">columns</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">exploded_df</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="p">(</span>
<span class="p">[</span><span class="n">label</span> <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span> <span class="k">else</span> <span class="p">(</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">),)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">id_vars</span><span class="p">]</span>
<span class="o">+</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">var_name</span><span class="p">]</span>
<span class="o">+</span> <span class="p">[(</span><span class="n">value_name</span><span class="p">,)]</span>
<span class="p">),</span>
<span class="p">)</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.stack"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.stack.html#pyspark.pandas.DataFrame.stack">[docs]</a> <span class="k">def</span> <span class="nf">stack</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">DataFrameOrSeries</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Stack the prescribed level(s) from columns to index.</span>
<span class="sd"> Return a reshaped DataFrame or Series having a multi-level</span>
<span class="sd"> index with one or more new inner-most levels compared to the current</span>
<span class="sd"> DataFrame. The new inner-most levels are created by pivoting the</span>
<span class="sd"> columns of the current dataframe:</span>
<span class="sd"> - if the columns have a single level, the output is a Series</span>
<span class="sd"> - if the columns have multiple levels, the new index</span>
<span class="sd"> level(s) is (are) taken from the prescribed level(s) and</span>
<span class="sd"> the output is a DataFrame.</span>
<span class="sd"> The new index levels are sorted.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame or Series</span>
<span class="sd"> Stacked dataframe or series.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.unstack : Unstack prescribed level(s) from index axis</span>
<span class="sd"> onto column axis.</span>
<span class="sd"> DataFrame.pivot : Reshape dataframe from long format to wide</span>
<span class="sd"> format.</span>
<span class="sd"> DataFrame.pivot_table : Create a spreadsheet-style pivot table</span>
<span class="sd"> as a DataFrame.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> The function is named by analogy with a collection of books</span>
<span class="sd"> being reorganized from being side by side on a horizontal</span>
<span class="sd"> position (the columns of the dataframe) to being stacked</span>
<span class="sd"> vertically on top of each other (in the index of the</span>
<span class="sd"> dataframe).</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> **Single level columns**</span>
<span class="sd"> &gt;&gt;&gt; df_single_level_cols = ps.DataFrame([[0, 1], [2, 3]],</span>
<span class="sd"> ... index=[&#39;cat&#39;, &#39;dog&#39;],</span>
<span class="sd"> ... columns=[&#39;weight&#39;, &#39;height&#39;])</span>
<span class="sd"> Stacking a dataframe with a single level column axis returns a Series:</span>
<span class="sd"> &gt;&gt;&gt; df_single_level_cols</span>
<span class="sd"> weight height</span>
<span class="sd"> cat 0 1</span>
<span class="sd"> dog 2 3</span>
<span class="sd"> &gt;&gt;&gt; df_single_level_cols.stack().sort_index()</span>
<span class="sd"> cat height 1</span>
<span class="sd"> weight 0</span>
<span class="sd"> dog height 3</span>
<span class="sd"> weight 2</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> **Multi level columns: simple case**</span>
<span class="sd"> &gt;&gt;&gt; multicol1 = pd.MultiIndex.from_tuples([(&#39;weight&#39;, &#39;kg&#39;),</span>
<span class="sd"> ... (&#39;weight&#39;, &#39;pounds&#39;)])</span>
<span class="sd"> &gt;&gt;&gt; df_multi_level_cols1 = ps.DataFrame([[1, 2], [2, 4]],</span>
<span class="sd"> ... index=[&#39;cat&#39;, &#39;dog&#39;],</span>
<span class="sd"> ... columns=multicol1)</span>
<span class="sd"> Stacking a dataframe with a multi-level column axis:</span>
<span class="sd"> &gt;&gt;&gt; df_multi_level_cols1 # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> weight</span>
<span class="sd"> kg pounds</span>
<span class="sd"> cat 1 2</span>
<span class="sd"> dog 2 4</span>
<span class="sd"> &gt;&gt;&gt; df_multi_level_cols1.stack().sort_index()</span>
<span class="sd"> weight</span>
<span class="sd"> cat kg 1</span>
<span class="sd"> pounds 2</span>
<span class="sd"> dog kg 2</span>
<span class="sd"> pounds 4</span>
<span class="sd"> **Missing values**</span>
<span class="sd"> &gt;&gt;&gt; multicol2 = pd.MultiIndex.from_tuples([(&#39;weight&#39;, &#39;kg&#39;),</span>
<span class="sd"> ... (&#39;height&#39;, &#39;m&#39;)])</span>
<span class="sd"> &gt;&gt;&gt; df_multi_level_cols2 = ps.DataFrame([[1.0, 2.0], [3.0, 4.0]],</span>
<span class="sd"> ... index=[&#39;cat&#39;, &#39;dog&#39;],</span>
<span class="sd"> ... columns=multicol2)</span>
<span class="sd"> It is common to have missing values when stacking a dataframe</span>
<span class="sd"> with multi-level columns, as the stacked dataframe typically</span>
<span class="sd"> has more values than the original dataframe. Missing values</span>
<span class="sd"> are filled with NaNs:</span>
<span class="sd"> &gt;&gt;&gt; df_multi_level_cols2</span>
<span class="sd"> weight height</span>
<span class="sd"> kg m</span>
<span class="sd"> cat 1.0 2.0</span>
<span class="sd"> dog 3.0 4.0</span>
<span class="sd"> &gt;&gt;&gt; df_multi_level_cols2.stack().sort_index()</span>
<span class="sd"> weight height</span>
<span class="sd"> cat kg 1.0 NaN</span>
<span class="sd"> m NaN 2.0</span>
<span class="sd"> dog kg 3.0 NaN</span>
<span class="sd"> m NaN 4.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">first_series</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span>
<span class="p">)</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">))</span>
<span class="p">)</span>
<span class="n">column_labels</span><span class="p">:</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Label</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">PySparkColumn</span><span class="p">]]</span> <span class="o">=</span> <span class="n">defaultdict</span><span class="p">(</span><span class="nb">dict</span><span class="p">)</span>
<span class="n">index_values</span> <span class="o">=</span> <span class="nb">set</span><span class="p">()</span>
<span class="n">should_returns_series</span> <span class="o">=</span> <span class="kc">False</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">new_label</span> <span class="o">=</span> <span class="n">label</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="nb">len</span><span class="p">(</span><span class="n">new_label</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">new_label</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">should_returns_series</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">value</span> <span class="o">=</span> <span class="n">label</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span>
<span class="n">scol</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">column_labels</span><span class="p">[</span><span class="n">new_label</span><span class="p">][</span><span class="n">value</span><span class="p">]</span> <span class="o">=</span> <span class="n">scol</span>
<span class="n">index_values</span><span class="o">.</span><span class="n">add</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="n">index_name</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</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="nb">len</span><span class="p">(</span><span class="n">column_label_names</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="p">[</span><span class="kc">None</span><span class="p">]</span>
<span class="n">index_column</span> <span class="o">=</span> <span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">)</span>
<span class="n">data_columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">]</span>
<span class="n">structs</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">value</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_column</span><span class="p">)],</span>
<span class="o">*</span><span class="p">[</span>
<span class="p">(</span>
<span class="n">column_labels</span><span class="p">[</span><span class="n">label</span><span class="p">][</span><span class="n">value</span><span class="p">]</span>
<span class="k">if</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">[</span><span class="n">label</span><span class="p">]</span>
<span class="k">else</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span>
<span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name</span><span class="p">)</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">data_columns</span><span class="p">)</span>
<span class="p">],</span>
<span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="k">for</span> <span class="n">value</span> <span class="ow">in</span> <span class="n">index_values</span>
<span class="p">]</span>
<span class="n">pairs</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span><span class="o">*</span><span class="n">structs</span><span class="p">))</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="s2">&quot;pairs&quot;</span><span class="p">,</span> <span class="n">pairs</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="o">+</span> <span class="p">[</span><span class="n">sdf</span><span class="p">[</span><span class="s2">&quot;pairs&quot;</span><span class="p">][</span><span class="n">index_column</span><span class="p">]</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">index_column</span><span class="p">)]</span>
<span class="o">+</span> <span class="p">[</span><span class="n">sdf</span><span class="p">[</span><span class="s2">&quot;pairs&quot;</span><span class="p">][</span><span class="n">name</span><span class="p">]</span><span class="o">.</span><span class="n">alias</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">data_columns</span><span class="p">]</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span>
<span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span> <span class="o">+</span> <span class="p">[</span><span class="n">index_column</span><span class="p">])</span>
<span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span> <span class="o">+</span> <span class="p">[</span><span class="n">index_name</span><span class="p">],</span>
<span class="n">index_fields</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span> <span class="o">+</span> <span class="p">[</span><span class="kc">None</span><span class="p">],</span>
<span class="n">column_labels</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">column_labels</span><span class="p">),</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">data_columns</span><span class="p">],</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="n">column_label_names</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">if</span> <span class="n">should_returns_series</span><span class="p">:</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">psdf</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.unstack"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.unstack.html#pyspark.pandas.DataFrame.unstack">[docs]</a> <span class="k">def</span> <span class="nf">unstack</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">DataFrameOrSeries</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Pivot the (necessarily hierarchical) index labels.</span>
<span class="sd"> Returns a DataFrame having a new level of column labels whose inner-most level</span>
<span class="sd"> consists of the pivoted index labels.</span>
<span class="sd"> If the index is not a MultiIndex, the output will be a Series.</span>
<span class="sd"> .. note:: If the index is a MultiIndex, the output DataFrame could be very wide, and</span>
<span class="sd"> it could cause a serious performance degradation since Spark partitions its row based.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series or DataFrame</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.pivot : Pivot a table based on column values.</span>
<span class="sd"> DataFrame.stack : Pivot a level of the column labels (inverse operation from unstack).</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&quot;A&quot;: {&quot;0&quot;: &quot;a&quot;, &quot;1&quot;: &quot;b&quot;, &quot;2&quot;: &quot;c&quot;},</span>
<span class="sd"> ... &quot;B&quot;: {&quot;0&quot;: &quot;1&quot;, &quot;1&quot;: &quot;3&quot;, &quot;2&quot;: &quot;5&quot;},</span>
<span class="sd"> ... &quot;C&quot;: {&quot;0&quot;: &quot;2&quot;, &quot;1&quot;: &quot;4&quot;, &quot;2&quot;: &quot;6&quot;}},</span>
<span class="sd"> ... columns=[&quot;A&quot;, &quot;B&quot;, &quot;C&quot;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C</span>
<span class="sd"> 0 a 1 2</span>
<span class="sd"> 1 b 3 4</span>
<span class="sd"> 2 c 5 6</span>
<span class="sd"> &gt;&gt;&gt; df.unstack().sort_index()</span>
<span class="sd"> A 0 a</span>
<span class="sd"> 1 b</span>
<span class="sd"> 2 c</span>
<span class="sd"> B 0 1</span>
<span class="sd"> 1 3</span>
<span class="sd"> 2 5</span>
<span class="sd"> C 0 2</span>
<span class="sd"> 1 4</span>
<span class="sd"> 2 6</span>
<span class="sd"> dtype: object</span>
<span class="sd"> &gt;&gt;&gt; df.columns = pd.MultiIndex.from_tuples([(&#39;X&#39;, &#39;A&#39;), (&#39;X&#39;, &#39;B&#39;), (&#39;Y&#39;, &#39;C&#39;)])</span>
<span class="sd"> &gt;&gt;&gt; df.unstack().sort_index()</span>
<span class="sd"> X A 0 a</span>
<span class="sd"> 1 b</span>
<span class="sd"> 2 c</span>
<span class="sd"> B 0 1</span>
<span class="sd"> 1 3</span>
<span class="sd"> 2 5</span>
<span class="sd"> Y C 0 2</span>
<span class="sd"> 1 4</span>
<span class="sd"> 2 6</span>
<span class="sd"> dtype: object</span>
<span class="sd"> For MultiIndex case:</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&quot;A&quot;: [&quot;a&quot;, &quot;b&quot;, &quot;c&quot;],</span>
<span class="sd"> ... &quot;B&quot;: [1, 3, 5],</span>
<span class="sd"> ... &quot;C&quot;: [2, 4, 6]},</span>
<span class="sd"> ... columns=[&quot;A&quot;, &quot;B&quot;, &quot;C&quot;])</span>
<span class="sd"> &gt;&gt;&gt; df = df.set_index(&#39;A&#39;, append=True)</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> B C</span>
<span class="sd"> A</span>
<span class="sd"> 0 a 1 2</span>
<span class="sd"> 1 b 3 4</span>
<span class="sd"> 2 c 5 6</span>
<span class="sd"> &gt;&gt;&gt; df.unstack().sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> B C</span>
<span class="sd"> A a b c a b c</span>
<span class="sd"> 0 1.0 NaN NaN 2.0 NaN NaN</span>
<span class="sd"> 1 NaN 3.0 NaN NaN 4.0 NaN</span>
<span class="sd"> 2 NaN NaN 5.0 NaN NaN 6.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">first_series</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="c1"># The index after `reset_index()` will never be used, so use &quot;distributed&quot; index</span>
<span class="c1"># as a dummy to avoid overhead.</span>
<span class="k">with</span> <span class="n">option_context</span><span class="p">(</span><span class="s2">&quot;compute.default_index_type&quot;</span><span class="p">,</span> <span class="s2">&quot;distributed&quot;</span><span class="p">):</span>
<span class="n">df</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">reset_index</span><span class="p">()</span>
<span class="n">index</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">[:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span> <span class="o">-</span> <span class="mi">1</span><span class="p">]</span>
<span class="n">columns</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">columns</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span> <span class="o">-</span> <span class="mi">1</span><span class="p">]</span>
<span class="n">df</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">pivot_table</span><span class="p">(</span>
<span class="n">index</span><span class="o">=</span><span class="n">index</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">columns</span><span class="p">,</span> <span class="n">values</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">aggfunc</span><span class="o">=</span><span class="s2">&quot;first&quot;</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">index_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">],</span>
<span class="n">index_fields</span><span class="o">=</span><span class="n">df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">[:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span> <span class="o">-</span> <span class="mi">1</span><span class="p">],</span>
<span class="n">column_label_names</span><span class="o">=</span><span class="p">(</span>
<span class="n">df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">[:</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span>
<span class="o">+</span> <span class="p">[</span>
<span class="kc">None</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> <span class="ow">is</span> <span class="kc">None</span>
<span class="k">else</span> <span class="n">df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span>
<span class="p">]</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">internal</span><span class="p">)</span>
<span class="c1"># TODO: Codes here are similar with melt. Should we deduplicate?</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">ser_name</span> <span class="o">=</span> <span class="n">SPARK_DEFAULT_SERIES_NAME</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="n">new_index_columns</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="n">i</span><span class="p">)</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">new_index_map</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">zip_longest</span><span class="p">(</span><span class="n">new_index_columns</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">,</span> <span class="p">[]))</span>
<span class="n">pairs</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">c</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> <span class="k">for</span> <span class="n">c</span><span class="p">,</span> <span class="n">name</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">idx</span><span class="p">,</span> <span class="n">new_index_columns</span><span class="p">)],</span>
<span class="o">*</span><span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">idx</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">ser_name</span><span class="p">)],</span>
<span class="p">)</span>
<span class="k">for</span> <span class="n">idx</span> <span class="ow">in</span> <span class="n">column_labels</span>
<span class="p">]</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">columns</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;pairs.</span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</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">new_index_columns</span><span class="p">[:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">]</span>
<span class="p">]</span> <span class="o">+</span> <span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;pairs.</span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="n">ser_name</span><span class="p">)]</span>
<span class="n">new_index_len</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">new_index_columns</span><span class="p">)</span>
<span class="n">existing_index_columns</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="p">(</span><span class="n">index_name</span><span class="p">,</span> <span class="n">index_field</span><span class="p">)</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span>
<span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="p">)</span>
<span class="p">):</span>
<span class="n">name</span> <span class="o">=</span> <span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="n">i</span> <span class="o">+</span> <span class="n">new_index_len</span><span class="p">)</span>
<span class="n">new_index_map</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">name</span><span class="p">,</span> <span class="n">index_name</span><span class="p">,</span> <span class="n">index_field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="n">name</span><span class="p">)))</span>
<span class="n">existing_index_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="n">i</span><span class="p">]</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">name</span><span class="p">))</span>
<span class="n">exploded_df</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="s2">&quot;pairs&quot;</span><span class="p">,</span> <span class="n">pairs</span><span class="p">)</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">existing_index_columns</span> <span class="o">+</span> <span class="n">columns</span><span class="p">)</span>
<span class="n">index_spark_column_names</span><span class="p">,</span> <span class="n">index_names</span><span class="p">,</span> <span class="n">index_fields</span> <span class="o">=</span> <span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">new_index_map</span><span class="p">)</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span>
<span class="n">DataFrame</span><span class="p">(</span>
<span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">exploded_df</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">exploded_df</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">index_spark_column_names</span>
<span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_names</span><span class="p">),</span>
<span class="n">index_fields</span><span class="o">=</span><span class="nb">list</span><span class="p">(</span><span class="n">index_fields</span><span class="p">),</span>
<span class="n">column_labels</span><span class="o">=</span><span class="p">[</span><span class="kc">None</span><span class="p">],</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46165): axis and **kwargs should be implemented.</span>
<div class="viewcode-block" id="DataFrame.all"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.all.html#pyspark.pandas.DataFrame.all">[docs]</a> <span class="k">def</span> <span class="nf">all</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="n">bool_only</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> <span class="n">skipna</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return whether all elements are True.</span>
<span class="sd"> Returns True unless there is at least one element within a series that is</span>
<span class="sd"> False or equivalent (e.g. zero or empty)</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> axis : {0 or &#39;index&#39;}, default 0</span>
<span class="sd"> Indicate which axis or axes should be reduced.</span>
<span class="sd"> * 0 / &#39;index&#39; : reduce the index, return a Series whose index is the</span>
<span class="sd"> original column labels.</span>
<span class="sd"> bool_only : bool, default None</span>
<span class="sd"> Include only boolean columns. If None, will attempt to use everything,</span>
<span class="sd"> then use only boolean data.</span>
<span class="sd"> skipna : boolean, default True</span>
<span class="sd"> Exclude NA values, such as None or numpy.NaN.</span>
<span class="sd"> If an entire row/column is NA values and `skipna` is True,</span>
<span class="sd"> then the result will be True, as for an empty row/column.</span>
<span class="sd"> If `skipna` is False, numpy.NaNs are treated as True because these are</span>
<span class="sd"> not equal to zero, Nones are treated as False.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> Create a dataframe from a dictionary.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({</span>
<span class="sd"> ... &#39;col1&#39;: [True, True, True],</span>
<span class="sd"> ... &#39;col2&#39;: [True, False, False],</span>
<span class="sd"> ... &#39;col3&#39;: [0, 0, 0],</span>
<span class="sd"> ... &#39;col4&#39;: [1, 2, 3],</span>
<span class="sd"> ... &#39;col5&#39;: [True, True, None],</span>
<span class="sd"> ... &#39;col6&#39;: [True, False, None]},</span>
<span class="sd"> ... columns=[&#39;col1&#39;, &#39;col2&#39;, &#39;col3&#39;, &#39;col4&#39;, &#39;col5&#39;, &#39;col6&#39;])</span>
<span class="sd"> Default behavior checks if column-wise values all return True.</span>
<span class="sd"> &gt;&gt;&gt; df.all()</span>
<span class="sd"> col1 True</span>
<span class="sd"> col2 False</span>
<span class="sd"> col3 False</span>
<span class="sd"> col4 True</span>
<span class="sd"> col5 True</span>
<span class="sd"> col6 False</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> Include NA values when set `skipna=False`.</span>
<span class="sd"> &gt;&gt;&gt; df[[&#39;col5&#39;, &#39;col6&#39;]].all(skipna=False)</span>
<span class="sd"> col5 False</span>
<span class="sd"> col6 False</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> Include only boolean columns when set `bool_only=True`.</span>
<span class="sd"> &gt;&gt;&gt; df.all(bool_only=True)</span>
<span class="sd"> col1 True</span>
<span class="sd"> col2 False</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</span><span class="p">)</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">if</span> <span class="n">bool_only</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_bool_column_labels</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">([],</span> <span class="n">dtype</span><span class="o">=</span><span class="nb">bool</span><span class="p">)</span>
<span class="n">applied</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">label</span><span class="p">),</span> <span class="n">NumericType</span><span class="p">)</span> <span class="ow">or</span> <span class="n">skipna</span><span class="p">:</span>
<span class="c1"># np.nan takes no effect to the result; None takes no effect if `skipna`</span>
<span class="n">all_col</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">min</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">coalesce</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;boolean&quot;</span><span class="p">),</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">True</span><span class="p">)))</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Take None as False when not `skipna`</span>
<span class="n">all_col</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">min</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">isNull</span><span class="p">(),</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">))</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;boolean&quot;</span><span class="p">)))</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">all_col</span><span class="o">.</span><span class="n">isNull</span><span class="p">(),</span> <span class="kc">True</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">all_col</span><span class="p">))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_result_aggregated</span><span class="p">(</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">applied</span><span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46166): axis, skipna and **kwargs should be implemented.</span>
<div class="viewcode-block" id="DataFrame.any"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.any.html#pyspark.pandas.DataFrame.any">[docs]</a> <span class="k">def</span> <span class="nf">any</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="n">bool_only</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return whether any element is True.</span>
<span class="sd"> Returns False unless there is at least one element within a series that is</span>
<span class="sd"> True or equivalent (e.g. non-zero or non-empty).</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> axis : {0 or &#39;index&#39;}, default 0</span>
<span class="sd"> Indicate which axis or axes should be reduced.</span>
<span class="sd"> * 0 / &#39;index&#39; : reduce the index, return a Series whose index is the</span>
<span class="sd"> original column labels.</span>
<span class="sd"> bool_only : bool, default None</span>
<span class="sd"> Include only boolean columns. If None, will attempt to use everything,</span>
<span class="sd"> then use only boolean data.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> Create a dataframe from a dictionary.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({</span>
<span class="sd"> ... &#39;col1&#39;: [False, False, False],</span>
<span class="sd"> ... &#39;col2&#39;: [True, False, False],</span>
<span class="sd"> ... &#39;col3&#39;: [0, 0, 1],</span>
<span class="sd"> ... &#39;col4&#39;: [0, 1, 2],</span>
<span class="sd"> ... &#39;col5&#39;: [False, False, None],</span>
<span class="sd"> ... &#39;col6&#39;: [True, False, None]},</span>
<span class="sd"> ... columns=[&#39;col1&#39;, &#39;col2&#39;, &#39;col3&#39;, &#39;col4&#39;, &#39;col5&#39;, &#39;col6&#39;])</span>
<span class="sd"> Default behavior checks if column-wise values all return True.</span>
<span class="sd"> &gt;&gt;&gt; df.any()</span>
<span class="sd"> col1 False</span>
<span class="sd"> col2 True</span>
<span class="sd"> col3 True</span>
<span class="sd"> col4 True</span>
<span class="sd"> col5 False</span>
<span class="sd"> col6 True</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> Include only boolean columns when set `bool_only=True`.</span>
<span class="sd"> &gt;&gt;&gt; df.any(bool_only=True)</span>
<span class="sd"> col1 False</span>
<span class="sd"> col2 True</span>
<span class="sd"> dtype: bool</span>
<span class="sd"> Returns empty Series when the DataFrame is empty.</span>
<span class="sd"> &gt;&gt;&gt; df[[]].any()</span>
<span class="sd"> Series([], dtype: bool)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</span><span class="p">)</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">if</span> <span class="n">bool_only</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_bool_column_labels</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">([],</span> <span class="n">dtype</span><span class="o">=</span><span class="nb">bool</span><span class="p">)</span>
<span class="n">applied</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_column_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">any_col</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">max</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">coalesce</span><span class="p">(</span><span class="n">scol</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="s2">&quot;boolean&quot;</span><span class="p">),</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">)))</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">any_col</span><span class="o">.</span><span class="n">isNull</span><span class="p">(),</span> <span class="kc">False</span><span class="p">)</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">any_col</span><span class="p">))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_result_aggregated</span><span class="p">(</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">applied</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_bool_column_labels</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Filter column labels of boolean columns (without None).</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">bool_column_labels</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="n">is_bool_dtype</span><span class="p">(</span><span class="n">psser</span><span class="p">):</span>
<span class="c1"># Rely on dtype rather than spark type because</span>
<span class="c1"># columns that consist of bools and Nones should be excluded</span>
<span class="c1"># if bool_only is True</span>
<span class="n">bool_column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">return</span> <span class="n">bool_column_labels</span>
<span class="k">def</span> <span class="nf">_result_aggregated</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">],</span> <span class="n">scols</span><span class="p">:</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">]</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Given aggregated Spark columns and respective column labels from the original</span>
<span class="sd"> pandas-on-Spark DataFrame, construct the result Series.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">first_series</span>
<span class="n">cols</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">result_scol_name</span> <span class="o">=</span> <span class="s2">&quot;value&quot;</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">applied_col</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">scols</span><span class="p">):</span>
<span class="n">cols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">col</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="n">i</span><span class="p">))</span> <span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">col</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">label</span><span class="p">)],</span>
<span class="o">*</span><span class="p">[</span><span class="n">applied_col</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">result_scol_name</span><span class="p">)],</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="c1"># Statements under this comment implement spark frame transformations as below:</span>
<span class="c1"># From:</span>
<span class="c1"># +-------------------------------------------------------------------------------------+</span>
<span class="c1"># |arrays |</span>
<span class="c1"># +-------------------------------------------------------------------------------------+</span>
<span class="c1"># |[{col1, true}, {col2, true}, {col3, false}, {col4, true}]|</span>
<span class="c1"># +-------------------------------------------------------------------------------------+</span>
<span class="c1"># To:</span>
<span class="c1"># +-------------+</span>
<span class="c1"># |col |</span>
<span class="c1"># +-------------+</span>
<span class="c1"># |{col1, true} |</span>
<span class="c1"># |{col2, true} |</span>
<span class="c1"># |{col3, false}|</span>
<span class="c1"># |{col4, true} |</span>
<span class="c1"># +-------------+</span>
<span class="c1"># To:</span>
<span class="c1"># +-----------------+-----+</span>
<span class="c1"># |__index_level_0__|value|</span>
<span class="c1"># +-----------------+-----+</span>
<span class="c1"># |col1 |true |</span>
<span class="c1"># |col2 |true |</span>
<span class="c1"># |col3 |false|</span>
<span class="c1"># |col4 |true |</span>
<span class="c1"># +-----------------+-----+</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span><span class="o">*</span><span class="n">cols</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="s2">&quot;arrays&quot;</span><span class="p">))</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;arrays&quot;</span><span class="p">))</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">selectExpr</span><span class="p">(</span><span class="s2">&quot;col.*&quot;</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span>
<span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">SPARK_INDEX_NAME_FORMAT</span><span class="p">(</span><span class="n">i</span><span class="p">))</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">)</span>
<span class="p">],</span>
<span class="n">index_names</span><span class="o">=</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span><span class="p">,</span>
<span class="n">column_labels</span><span class="o">=</span><span class="p">[</span><span class="kc">None</span><span class="p">],</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">result_scol_name</span><span class="p">)],</span>
<span class="p">)</span>
<span class="c1"># (cont.) The result Series should look as below:</span>
<span class="c1"># col1 False</span>
<span class="c1"># col2 True</span>
<span class="c1"># col3 True</span>
<span class="c1"># col4 True</span>
<span class="c1"># dtype: bool</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">))</span>
<span class="c1"># TODO(SPARK-46167): add axis, pct, na_option parameter</span>
<div class="viewcode-block" id="DataFrame.rank"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rank.html#pyspark.pandas.DataFrame.rank">[docs]</a> <span class="k">def</span> <span class="nf">rank</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">method</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;average&quot;</span><span class="p">,</span> <span class="n">ascending</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span> <span class="n">numeric_only</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute numerical data ranks (1 through n) along axis. Equal values are</span>
<span class="sd"> assigned a rank that is the average of the ranks of those values.</span>
<span class="sd"> .. note:: the current implementation of rank uses Spark&#39;s Window without</span>
<span class="sd"> specifying partition specification. This leads to moving all data into</span>
<span class="sd"> a single partition in a single machine and could cause serious</span>
<span class="sd"> performance degradation. Avoid this method with very large datasets.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> method : {&#39;average&#39;, &#39;min&#39;, &#39;max&#39;, &#39;first&#39;, &#39;dense&#39;}</span>
<span class="sd"> * average: average rank of group</span>
<span class="sd"> * min: lowest rank in group</span>
<span class="sd"> * max: highest rank in group</span>
<span class="sd"> * first: ranks assigned in order they appear in the array</span>
<span class="sd"> * dense: like &#39;min&#39;, but rank always increases by 1 between groups</span>
<span class="sd"> ascending : boolean, default True</span>
<span class="sd"> False for ranks by high (1) to low (N)</span>
<span class="sd"> numeric_only : bool, default False</span>
<span class="sd"> For DataFrame objects, rank only numeric columns if set to True.</span>
<span class="sd"> .. versionchanged:: 4.0.0</span>
<span class="sd"> The default value of ``numeric_only`` is now ``False``.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> ranks : same type as caller</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [1, 2, 2, 3], &#39;B&#39;: [4, 3, 2, 1]}, columns=[&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 4</span>
<span class="sd"> 1 2 3</span>
<span class="sd"> 2 2 2</span>
<span class="sd"> 3 3 1</span>
<span class="sd"> &gt;&gt;&gt; df.rank().sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1.0 4.0</span>
<span class="sd"> 1 2.5 3.0</span>
<span class="sd"> 2 2.5 2.0</span>
<span class="sd"> 3 4.0 1.0</span>
<span class="sd"> If method is set to &#39;min&#39;, it uses lowest rank in group.</span>
<span class="sd"> &gt;&gt;&gt; df.rank(method=&#39;min&#39;).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1.0 4.0</span>
<span class="sd"> 1 2.0 3.0</span>
<span class="sd"> 2 2.0 2.0</span>
<span class="sd"> 3 4.0 1.0</span>
<span class="sd"> If method is set to &#39;max&#39;, it uses highest rank in group.</span>
<span class="sd"> &gt;&gt;&gt; df.rank(method=&#39;max&#39;).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1.0 4.0</span>
<span class="sd"> 1 3.0 3.0</span>
<span class="sd"> 2 3.0 2.0</span>
<span class="sd"> 3 4.0 1.0</span>
<span class="sd"> If method is set to &#39;dense&#39;, it leaves no gaps in group.</span>
<span class="sd"> &gt;&gt;&gt; df.rank(method=&#39;dense&#39;).sort_index()</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1.0 4.0</span>
<span class="sd"> 1 2.0 3.0</span>
<span class="sd"> 2 2.0 2.0</span>
<span class="sd"> 3 3.0 1.0</span>
<span class="sd"> If numeric_only is set to &#39;True&#39;, rank only numeric columns.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [1, 2, 2, 3], &#39;B&#39;: [&#39;a&#39;, &#39;b&#39;, &#39;d&#39;, &#39;c&#39;]}, columns= [&#39;A&#39;, &#39;B&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 a</span>
<span class="sd"> 1 2 b</span>
<span class="sd"> 2 2 d</span>
<span class="sd"> 3 3 c</span>
<span class="sd"> &gt;&gt;&gt; df.rank(numeric_only=True)</span>
<span class="sd"> A</span>
<span class="sd"> 0 1.0</span>
<span class="sd"> 1 2.5</span>
<span class="sd"> 2 2.5</span>
<span class="sd"> 3 4.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">numeric_only</span><span class="p">:</span>
<span class="n">numeric_col_names</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">)):</span>
<span class="n">numeric_col_names</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">name</span><span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="p">[</span><span class="n">numeric_col_names</span><span class="p">]</span> <span class="k">if</span> <span class="n">numeric_only</span> <span class="k">else</span> <span class="bp">self</span>
<span class="k">return</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">psser</span><span class="p">:</span> <span class="n">psser</span><span class="o">.</span><span class="n">_rank</span><span class="p">(</span><span class="n">method</span><span class="o">=</span><span class="n">method</span><span class="p">,</span> <span class="n">ascending</span><span class="o">=</span><span class="n">ascending</span><span class="p">),</span> <span class="n">should_resolve</span><span class="o">=</span><span class="kc">True</span>
<span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.filter"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.filter.html#pyspark.pandas.DataFrame.filter">[docs]</a> <span class="k">def</span> <span class="nf">filter</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">items</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">like</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="n">regex</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="n">axis</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Axis</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Subset rows or columns of dataframe according to labels in</span>
<span class="sd"> the specified index.</span>
<span class="sd"> Note that this routine does not filter a dataframe on its</span>
<span class="sd"> contents. The filter is applied to the labels of the index.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> items : list-like</span>
<span class="sd"> Keep labels from axis which are in items.</span>
<span class="sd"> like : string</span>
<span class="sd"> Keep labels from axis for which &quot;like in label == True&quot;.</span>
<span class="sd"> regex : string (regular expression)</span>
<span class="sd"> Keep labels from axis for which re.search(regex, label) == True.</span>
<span class="sd"> axis : int or string axis name</span>
<span class="sd"> The axis to filter on. By default this is the info axis,</span>
<span class="sd"> &#39;index&#39; for Series, &#39;columns&#39; for DataFrame.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> same type as input object</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.loc</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> The ``items``, ``like``, and ``regex`` parameters are</span>
<span class="sd"> enforced to be mutually exclusive.</span>
<span class="sd"> ``axis`` defaults to the info axis that is used when indexing</span>
<span class="sd"> with ``[]``.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(np.array(([1, 2, 3], [4, 5, 6])),</span>
<span class="sd"> ... index=[&#39;mouse&#39;, &#39;rabbit&#39;],</span>
<span class="sd"> ... columns=[&#39;one&#39;, &#39;two&#39;, &#39;three&#39;])</span>
<span class="sd"> &gt;&gt;&gt; # select columns by name</span>
<span class="sd"> &gt;&gt;&gt; df.filter(items=[&#39;one&#39;, &#39;three&#39;])</span>
<span class="sd"> one three</span>
<span class="sd"> mouse 1 3</span>
<span class="sd"> rabbit 4 6</span>
<span class="sd"> &gt;&gt;&gt; # select columns by regular expression</span>
<span class="sd"> &gt;&gt;&gt; df.filter(regex=&#39;e$&#39;, axis=1)</span>
<span class="sd"> one three</span>
<span class="sd"> mouse 1 3</span>
<span class="sd"> rabbit 4 6</span>
<span class="sd"> &gt;&gt;&gt; # select rows containing &#39;bbi&#39;</span>
<span class="sd"> &gt;&gt;&gt; df.filter(like=&#39;bbi&#39;, axis=0)</span>
<span class="sd"> one two three</span>
<span class="sd"> rabbit 4 5 6</span>
<span class="sd"> For a Series,</span>
<span class="sd"> &gt;&gt;&gt; # select rows by name</span>
<span class="sd"> &gt;&gt;&gt; df.one.filter(items=[&#39;rabbit&#39;])</span>
<span class="sd"> rabbit 4</span>
<span class="sd"> Name: one, dtype: int64</span>
<span class="sd"> &gt;&gt;&gt; # select rows by regular expression</span>
<span class="sd"> &gt;&gt;&gt; df.one.filter(regex=&#39;e$&#39;)</span>
<span class="sd"> mouse 1</span>
<span class="sd"> Name: one, dtype: int64</span>
<span class="sd"> &gt;&gt;&gt; # select rows containing &#39;bbi&#39;</span>
<span class="sd"> &gt;&gt;&gt; df.one.filter(like=&#39;bbi&#39;)</span>
<span class="sd"> rabbit 4</span>
<span class="sd"> Name: one, dtype: int64</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">sum</span><span class="p">(</span><span class="n">x</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="p">(</span><span class="n">items</span><span class="p">,</span> <span class="n">like</span><span class="p">,</span> <span class="n">regex</span><span class="p">))</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Keyword arguments `items`, `like`, or `regex` &quot;</span> <span class="s2">&quot;are mutually exclusive&quot;</span>
<span class="p">)</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">,</span> <span class="n">none_axis</span><span class="o">=</span><span class="mi">1</span><span class="p">)</span>
<span class="n">index_scols</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="k">if</span> <span class="n">items</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">items</span><span class="p">):</span>
<span class="n">items</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">items</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;items should be a list-like object.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">index_scols</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="nb">len</span><span class="p">(</span><span class="n">items</span><span class="p">)</span> <span class="o">&lt;=</span> <span class="n">ps</span><span class="o">.</span><span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.isin_limit&quot;</span><span class="p">):</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">index_scols</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">isin</span><span class="p">([</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">item</span><span class="p">)</span> <span class="k">for</span> <span class="n">item</span> <span class="ow">in</span> <span class="n">items</span><span class="p">])</span>
<span class="n">result</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">col</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">item_sdf_col</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="p">,</span> <span class="s2">&quot;__item__&quot;</span>
<span class="p">)</span>
<span class="n">item_sdf</span> <span class="o">=</span> <span class="n">default_session</span><span class="p">()</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span>
<span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">({</span><span class="n">item_sdf_col</span><span class="p">:</span> <span class="n">items</span><span class="p">})</span>
<span class="p">)</span>
<span class="n">joined_sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">join</span><span class="p">(</span>
<span class="n">other</span><span class="o">=</span><span class="n">F</span><span class="o">.</span><span class="n">broadcast</span><span class="p">(</span><span class="n">item_sdf</span><span class="p">),</span>
<span class="n">on</span><span class="o">=</span><span class="p">(</span><span class="n">index_scols</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">==</span> <span class="n">scol_for</span><span class="p">(</span><span class="n">item_sdf</span><span class="p">,</span> <span class="n">item_sdf_col</span><span class="p">)),</span>
<span class="n">how</span><span class="o">=</span><span class="s2">&quot;semi&quot;</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">joined_sdf</span><span class="p">))</span>
<span class="n">result</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">name</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">return</span> <span class="n">result</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># for multi-index</span>
<span class="n">col</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">for</span> <span class="n">item</span> <span class="ow">in</span> <span class="n">items</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">item</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Unsupported type </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">item</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">))</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">item</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;The item should not be empty.&quot;</span><span class="p">)</span>
<span class="n">midx_col</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">element</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">item</span><span class="p">):</span>
<span class="k">if</span> <span class="n">midx_col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">midx_col</span> <span class="o">=</span> <span class="n">index_scols</span><span class="p">[</span><span class="n">i</span><span class="p">]</span> <span class="o">==</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">element</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">midx_col</span> <span class="o">=</span> <span class="n">midx_col</span> <span class="o">&amp;</span> <span class="p">(</span><span class="n">index_scols</span><span class="p">[</span><span class="n">i</span><span class="p">]</span> <span class="o">==</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">element</span><span class="p">))</span>
<span class="k">if</span> <span class="n">col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">midx_col</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">col</span> <span class="o">|</span> <span class="n">midx_col</span>
<span class="n">result</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">col</span><span class="p">))</span>
<span class="n">result</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span> <span class="o">=</span> <span class="p">[</span><span class="kc">None</span><span class="p">]</span> <span class="o">*</span> <span class="n">result</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">nlevels</span>
<span class="k">return</span> <span class="n">result</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">items</span><span class="p">]</span>
<span class="k">elif</span> <span class="n">like</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">for</span> <span class="n">index_scol</span> <span class="ow">in</span> <span class="n">index_scols</span><span class="p">:</span>
<span class="k">if</span> <span class="n">col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">index_scol</span><span class="o">.</span><span class="n">contains</span><span class="p">(</span><span class="n">like</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">col</span> <span class="o">|</span> <span class="n">index_scol</span><span class="o">.</span><span class="n">contains</span><span class="p">(</span><span class="n">like</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">col</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">output_labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">label</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span> <span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="n">like</span> <span class="ow">in</span> <span class="n">i</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">label</span><span class="p">)]</span>
<span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">output_labels</span><span class="p">]</span>
<span class="k">elif</span> <span class="n">regex</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">for</span> <span class="n">index_scol</span> <span class="ow">in</span> <span class="n">index_scols</span><span class="p">:</span>
<span class="k">if</span> <span class="n">col</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">index_scol</span><span class="o">.</span><span class="n">rlike</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">col</span> <span class="o">=</span> <span class="n">col</span> <span class="o">|</span> <span class="n">index_scol</span><span class="o">.</span><span class="n">rlike</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">col</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="n">matcher</span> <span class="o">=</span> <span class="n">re</span><span class="o">.</span><span class="n">compile</span><span class="p">(</span><span class="n">regex</span><span class="p">)</span>
<span class="n">output_labels</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">label</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="n">matcher</span><span class="o">.</span><span class="n">search</span><span class="p">(</span><span class="n">i</span><span class="p">)</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">label</span><span class="p">)</span>
<span class="p">]</span>
<span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">output_labels</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Must pass either `items`, `like`, or `regex`&quot;</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.rename"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rename.html#pyspark.pandas.DataFrame.rename">[docs]</a> <span class="k">def</span> <span class="nf">rename</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">mapper</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Dict</span><span class="p">,</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Any</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Dict</span><span class="p">,</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Any</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Dict</span><span class="p">,</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Any</span><span class="p">]]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="s2">&quot;index&quot;</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">level</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">errors</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;ignore&quot;</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Alter axes labels.</span>
<span class="sd"> Function / dict values must be unique (1-to-1). Labels not contained in a dict / Series</span>
<span class="sd"> will be left as-is. Extra labels listed don’t throw an error.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> mapper : dict-like or function</span>
<span class="sd"> Dict-like or functions transformations to apply to that axis’ values.</span>
<span class="sd"> Use either `mapper` and `axis` to specify the axis to target with `mapper`, or `index`</span>
<span class="sd"> and `columns`.</span>
<span class="sd"> index : dict-like or function</span>
<span class="sd"> Alternative to specifying axis (&quot;mapper, axis=0&quot; is equivalent to &quot;index=mapper&quot;).</span>
<span class="sd"> columns : dict-like or function</span>
<span class="sd"> Alternative to specifying axis (&quot;mapper, axis=1&quot; is equivalent to &quot;columns=mapper&quot;).</span>
<span class="sd"> axis : int or str, default &#39;index&#39;</span>
<span class="sd"> Axis to target with mapper. Can be either the axis name (&#39;index&#39;, &#39;columns&#39;) or</span>
<span class="sd"> number (0, 1).</span>
<span class="sd"> inplace : bool, default False</span>
<span class="sd"> Whether to return a new DataFrame.</span>
<span class="sd"> level : int or level name, default None</span>
<span class="sd"> In case of a MultiIndex, only rename labels in the specified level.</span>
<span class="sd"> errors : {&#39;ignore&#39;, &#39;raise&#39;}, default &#39;ignore&#39;</span>
<span class="sd"> If &#39;raise&#39;, raise a `KeyError` when a dict-like `mapper`, `index`, or `columns`</span>
<span class="sd"> contains labels that are not present in the Index being transformed. If &#39;ignore&#39;,</span>
<span class="sd"> existing keys will be renamed, and extra keys will be ignored.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame with the renamed axis labels.</span>
<span class="sd"> Raises</span>
<span class="sd"> ------</span>
<span class="sd"> `KeyError`</span>
<span class="sd"> If any of the labels is not found in the selected axis and &quot;errors=&#39;raise&#39;&quot;.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; psdf1 = ps.DataFrame({&quot;A&quot;: [1, 2, 3], &quot;B&quot;: [4, 5, 6]})</span>
<span class="sd"> &gt;&gt;&gt; psdf1.rename(columns={&quot;A&quot;: &quot;a&quot;, &quot;B&quot;: &quot;c&quot;}) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> a c</span>
<span class="sd"> 0 1 4</span>
<span class="sd"> 1 2 5</span>
<span class="sd"> 2 3 6</span>
<span class="sd"> &gt;&gt;&gt; psdf1.rename(index={1: 10, 2: 20}) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 4</span>
<span class="sd"> 10 2 5</span>
<span class="sd"> 20 3 6</span>
<span class="sd"> &gt;&gt;&gt; psdf1.rename(columns={&quot;A&quot;: &quot;a&quot;, &quot;C&quot;: &quot;c&quot;}, errors=&quot;raise&quot;)</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> KeyError: &#39;Index include value which is not in the `mapper`&#39;</span>
<span class="sd"> &gt;&gt;&gt; def str_lower(s) -&gt; str:</span>
<span class="sd"> ... return str.lower(s)</span>
<span class="sd"> &gt;&gt;&gt; psdf1.rename(str_lower, axis=&#39;columns&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 4</span>
<span class="sd"> 1 2 5</span>
<span class="sd"> 2 3 6</span>
<span class="sd"> &gt;&gt;&gt; def mul10(x) -&gt; int:</span>
<span class="sd"> ... return x * 10</span>
<span class="sd"> &gt;&gt;&gt; psdf1.rename(mul10, axis=&#39;index&#39;) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 4</span>
<span class="sd"> 10 2 5</span>
<span class="sd"> 20 3 6</span>
<span class="sd"> &gt;&gt;&gt; idx = pd.MultiIndex.from_tuples([(&#39;X&#39;, &#39;A&#39;), (&#39;X&#39;, &#39;B&#39;), (&#39;Y&#39;, &#39;C&#39;), (&#39;Y&#39;, &#39;D&#39;)])</span>
<span class="sd"> &gt;&gt;&gt; psdf2 = ps.DataFrame([[1, 2, 3, 4], [5, 6, 7, 8]], columns=idx)</span>
<span class="sd"> &gt;&gt;&gt; psdf2.rename(columns=str_lower, level=0) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> x y</span>
<span class="sd"> A B C D</span>
<span class="sd"> 0 1 2 3 4</span>
<span class="sd"> 1 5 6 7 8</span>
<span class="sd"> &gt;&gt;&gt; psdf3 = ps.DataFrame([[1, 2], [3, 4], [5, 6], [7, 8]], index=idx, columns=list(&#39;ab&#39;))</span>
<span class="sd"> &gt;&gt;&gt; psdf3.rename(index=str_lower) # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> a b</span>
<span class="sd"> x a 1 2</span>
<span class="sd"> b 3 4</span>
<span class="sd"> y c 5 6</span>
<span class="sd"> d 7 8</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">gen_mapper_fn</span><span class="p">(</span>
<span class="n">mapper</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Dict</span><span class="p">,</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Any</span><span class="p">]],</span> <span class="n">skip_return_type</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Tuple</span><span class="p">[</span><span class="n">Callable</span><span class="p">[[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Any</span><span class="p">],</span> <span class="n">Dtype</span><span class="p">,</span> <span class="n">DataType</span><span class="p">]:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">mapper</span><span class="p">,</span> <span class="nb">dict</span><span class="p">):</span>
<span class="n">mapper_dict</span> <span class="o">=</span> <span class="n">mapper</span>
<span class="n">type_set</span> <span class="o">=</span> <span class="nb">set</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="nb">type</span><span class="p">(</span><span class="n">x</span><span class="p">),</span> <span class="n">mapper_dict</span><span class="o">.</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">type_set</span><span class="p">)</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Mapper dict should have the same value type.&quot;</span><span class="p">)</span>
<span class="n">dtype</span><span class="p">,</span> <span class="n">spark_return_type</span> <span class="o">=</span> <span class="n">pandas_on_spark_type</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="n">type_set</span><span class="p">)[</span><span class="mi">0</span><span class="p">])</span>
<span class="k">def</span> <span class="nf">mapper_fn</span><span class="p">(</span><span class="n">x</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Any</span><span class="p">:</span>
<span class="k">if</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">mapper_dict</span><span class="p">:</span>
<span class="k">return</span> <span class="n">mapper_dict</span><span class="p">[</span><span class="n">x</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="n">errors</span> <span class="o">==</span> <span class="s2">&quot;raise&quot;</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="s2">&quot;Index include value which is not in the `mapper`&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">x</span>
<span class="k">return</span> <span class="n">mapper_fn</span><span class="p">,</span> <span class="n">dtype</span><span class="p">,</span> <span class="n">spark_return_type</span>
<span class="k">elif</span> <span class="nb">callable</span><span class="p">(</span><span class="n">mapper</span><span class="p">):</span>
<span class="n">mapper_callable</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">Callable</span><span class="p">,</span> <span class="n">mapper</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">mapper_fn</span><span class="p">(</span><span class="n">x</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Any</span><span class="p">:</span>
<span class="k">return</span> <span class="n">mapper_callable</span><span class="p">(</span><span class="n">x</span><span class="p">)</span>
<span class="k">if</span> <span class="n">skip_return_type</span><span class="p">:</span>
<span class="k">return</span> <span class="n">mapper_fn</span><span class="p">,</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">return_type</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">ScalarType</span><span class="p">,</span> <span class="n">infer_return_type</span><span class="p">(</span><span class="n">mapper</span><span class="p">))</span>
<span class="n">dtype</span> <span class="o">=</span> <span class="n">return_type</span><span class="o">.</span><span class="n">dtype</span>
<span class="n">spark_return_type</span> <span class="o">=</span> <span class="n">return_type</span><span class="o">.</span><span class="n">spark_type</span>
<span class="k">return</span> <span class="n">mapper_fn</span><span class="p">,</span> <span class="n">dtype</span><span class="p">,</span> <span class="n">spark_return_type</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;`mapper` or `index` or `columns` should be &quot;</span>
<span class="s2">&quot;either dict-like or function type.&quot;</span>
<span class="p">)</span>
<span class="n">index_mapper_fn</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">index_mapper_ret_stype</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">columns_mapper_fn</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">mapper</span><span class="p">:</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">index_mapper_fn</span><span class="p">,</span> <span class="n">index_mapper_ret_dtype</span><span class="p">,</span> <span class="n">index_mapper_ret_stype</span> <span class="o">=</span> <span class="n">gen_mapper_fn</span><span class="p">(</span>
<span class="n">mapper</span>
<span class="p">)</span>
<span class="k">elif</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">columns_mapper_fn</span><span class="p">,</span> <span class="n">_</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">gen_mapper_fn</span><span class="p">(</span><span class="n">mapper</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="n">index</span><span class="p">:</span>
<span class="n">index_mapper_fn</span><span class="p">,</span> <span class="n">index_mapper_ret_dtype</span><span class="p">,</span> <span class="n">index_mapper_ret_stype</span> <span class="o">=</span> <span class="n">gen_mapper_fn</span><span class="p">(</span>
<span class="n">index</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">columns</span><span class="p">:</span>
<span class="n">columns_mapper_fn</span><span class="p">,</span> <span class="n">_</span><span class="p">,</span> <span class="n">_</span> <span class="o">=</span> <span class="n">gen_mapper_fn</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="n">skip_return_type</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">index</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">columns</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Either `index` or `columns` should be provided.&quot;</span><span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="k">if</span> <span class="n">index_mapper_fn</span><span class="p">:</span>
<span class="c1"># rename index labels, if `level` is None, rename all index columns, otherwise only</span>
<span class="c1"># rename the corresponding level index.</span>
<span class="c1"># implement this by transform the underlying spark dataframe,</span>
<span class="c1"># Example:</span>
<span class="c1"># suppose the psdf index column in underlying spark dataframe is &quot;index_0&quot;, &quot;index_1&quot;,</span>
<span class="c1"># if rename level 0 index labels, will do:</span>
<span class="c1"># ``psdf._sdf.withColumn(&quot;index_0&quot;, mapper_fn_udf(col(&quot;index_0&quot;))``</span>
<span class="c1"># if rename all index labels (`level` is None), then will do:</span>
<span class="c1"># ```</span>
<span class="c1"># psdf._sdf.withColumn(&quot;index_0&quot;, mapper_fn_udf(col(&quot;index_0&quot;))</span>
<span class="c1"># .withColumn(&quot;index_1&quot;, mapper_fn_udf(col(&quot;index_1&quot;))</span>
<span class="c1"># ```</span>
<span class="n">index_columns</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_column_names</span>
<span class="n">num_indices</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">index_columns</span><span class="p">)</span>
<span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="p">(</span><span class="n">level</span> <span class="o">&lt;</span> <span class="mi">0</span> <span class="ow">or</span> <span class="n">level</span> <span class="o">&gt;=</span> <span class="n">num_indices</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;level should be an integer between [0, </span><span class="si">%s</span><span class="s2">)&quot;</span> <span class="o">%</span> <span class="n">num_indices</span><span class="p">)</span>
<span class="nd">@pandas_udf</span><span class="p">(</span><span class="n">returnType</span><span class="o">=</span><span class="n">index_mapper_ret_stype</span><span class="p">)</span> <span class="c1"># type: ignore[call-overload]</span>
<span class="k">def</span> <span class="nf">index_mapper_udf</span><span class="p">(</span><span class="n">s</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">:</span>
<span class="k">return</span> <span class="n">s</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="n">index_mapper_fn</span><span class="p">)</span>
<span class="n">index_spark_columns</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">index_fields</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_fields</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">num_indices</span><span class="p">):</span>
<span class="n">index_spark_columns</span><span class="p">[</span><span class="n">i</span><span class="p">]</span> <span class="o">=</span> <span class="n">index_mapper_udf</span><span class="p">(</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="n">i</span><span class="p">])</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span>
<span class="n">index_columns</span><span class="p">[</span><span class="n">i</span><span class="p">]</span>
<span class="p">)</span>
<span class="n">index_fields</span><span class="p">[</span><span class="n">i</span><span class="p">]</span> <span class="o">=</span> <span class="n">index_fields</span><span class="p">[</span><span class="n">i</span><span class="p">]</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">dtype</span><span class="o">=</span><span class="n">index_mapper_ret_dtype</span><span class="p">,</span>
<span class="n">spark_type</span><span class="o">=</span><span class="n">index_mapper_ret_stype</span><span class="p">,</span>
<span class="n">nullable</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">index_spark_columns</span><span class="p">[</span><span class="n">level</span><span class="p">]</span> <span class="o">=</span> <span class="n">index_mapper_udf</span><span class="p">(</span><span class="n">index_spark_columns</span><span class="p">[</span><span class="n">level</span><span class="p">])</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span>
<span class="n">index_columns</span><span class="p">[</span><span class="n">level</span><span class="p">]</span>
<span class="p">)</span>
<span class="n">index_fields</span><span class="p">[</span><span class="n">level</span><span class="p">]</span> <span class="o">=</span> <span class="n">index_fields</span><span class="p">[</span><span class="n">level</span><span class="p">]</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">dtype</span><span class="o">=</span><span class="n">index_mapper_ret_dtype</span><span class="p">,</span>
<span class="n">spark_type</span><span class="o">=</span><span class="n">index_mapper_ret_stype</span><span class="p">,</span>
<span class="n">nullable</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="n">index_spark_columns</span><span class="p">,</span> <span class="n">index_fields</span><span class="o">=</span><span class="n">index_fields</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">columns_mapper_fn</span><span class="p">:</span>
<span class="c1"># rename column name.</span>
<span class="c1"># Will modify the `_internal._column_labels` and transform underlying spark dataframe</span>
<span class="c1"># to the same column name with `_internal._column_labels`.</span>
<span class="k">if</span> <span class="n">level</span><span class="p">:</span>
<span class="k">if</span> <span class="n">level</span> <span class="o">&lt;</span> <span class="mi">0</span> <span class="ow">or</span> <span class="n">level</span> <span class="o">&gt;=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;level should be an integer between [0, column_labels_level)&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">gen_new_column_labels_entry</span><span class="p">(</span><span class="n">column_labels_entry</span><span class="p">:</span> <span class="n">Label</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Label</span><span class="p">:</span>
<span class="k">if</span> <span class="n">level</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="c1"># rename all level columns</span>
<span class="k">return</span> <span class="nb">tuple</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="n">columns_mapper_fn</span><span class="p">,</span> <span class="n">column_labels_entry</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># only rename specified level column</span>
<span class="n">entry_list</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">column_labels_entry</span><span class="p">)</span>
<span class="n">entry_list</span><span class="p">[</span><span class="n">level</span><span class="p">]</span> <span class="o">=</span> <span class="n">columns_mapper_fn</span><span class="p">(</span><span class="n">entry_list</span><span class="p">[</span><span class="n">level</span><span class="p">])</span>
<span class="k">return</span> <span class="nb">tuple</span><span class="p">(</span><span class="n">entry_list</span><span class="p">)</span>
<span class="n">new_column_labels</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="n">gen_new_column_labels_entry</span><span class="p">,</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">))</span>
<span class="n">new_data_pssers</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">old_label</span><span class="p">)</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">new_label</span><span class="p">)</span>
<span class="k">for</span> <span class="n">old_label</span><span class="p">,</span> <span class="n">new_label</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="n">new_column_labels</span><span class="p">)</span>
<span class="p">]</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">new_data_pssers</span><span class="p">))</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">psdf</span></div>
<div class="viewcode-block" id="DataFrame.rename_axis"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.rename_axis.html#pyspark.pandas.DataFrame.rename_axis">[docs]</a> <span class="k">def</span> <span class="nf">rename_axis</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">mapper</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Any</span><span class="p">],</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Name</span><span class="p">],</span> <span class="n">Any</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">index</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Any</span><span class="p">],</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Name</span><span class="p">],</span> <span class="n">Any</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">columns</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Any</span><span class="p">],</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Name</span><span class="p">],</span> <span class="n">Any</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Axis</span><span class="p">]</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="n">inplace</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Set the name of the axis for the index or columns.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> mapper : scalar, list-like, optional</span>
<span class="sd"> A scalar, list-like, dict-like or functions transformations to</span>
<span class="sd"> apply to the axis name attribute.</span>
<span class="sd"> index, columns : scalar, list-like, dict-like or function, optional</span>
<span class="sd"> A scalar, list-like, dict-like or functions transformations to</span>
<span class="sd"> apply to that axis&#39; values.</span>
<span class="sd"> Use either ``mapper`` and ``axis`` to</span>
<span class="sd"> specify the axis to target with ``mapper``, or ``index``</span>
<span class="sd"> and/or ``columns``.</span>
<span class="sd"> axis : {0 or &#39;index&#39;, 1 or &#39;columns&#39;}, default 0</span>
<span class="sd"> The axis to rename.</span>
<span class="sd"> inplace : bool, default False</span>
<span class="sd"> Modifies the object directly, instead of creating a new DataFrame.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame, or None if `inplace` is True.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.rename : Alter Series index labels or name.</span>
<span class="sd"> DataFrame.rename : Alter DataFrame index labels or name.</span>
<span class="sd"> Index.rename : Set new names on index.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> ``DataFrame.rename_axis`` supports two calling conventions</span>
<span class="sd"> * ``(index=index_mapper, columns=columns_mapper, ...)``</span>
<span class="sd"> * ``(mapper, axis={&#39;index&#39;, &#39;columns&#39;}, ...)``</span>
<span class="sd"> The first calling convention will only modify the names of</span>
<span class="sd"> the index and/or the names of the Index object that is the columns.</span>
<span class="sd"> The second calling convention will modify the names of the</span>
<span class="sd"> corresponding index specified by axis.</span>
<span class="sd"> We *highly* recommend using keyword arguments to clarify your</span>
<span class="sd"> intent.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&quot;num_legs&quot;: [4, 4, 2],</span>
<span class="sd"> ... &quot;num_arms&quot;: [0, 0, 2]},</span>
<span class="sd"> ... index=[&quot;dog&quot;, &quot;cat&quot;, &quot;monkey&quot;],</span>
<span class="sd"> ... columns=[&quot;num_legs&quot;, &quot;num_arms&quot;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> num_legs num_arms</span>
<span class="sd"> dog 4 0</span>
<span class="sd"> cat 4 0</span>
<span class="sd"> monkey 2 2</span>
<span class="sd"> &gt;&gt;&gt; df = df.rename_axis(&quot;animal&quot;).sort_index()</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs num_arms</span>
<span class="sd"> animal</span>
<span class="sd"> cat 4 0</span>
<span class="sd"> dog 4 0</span>
<span class="sd"> monkey 2 2</span>
<span class="sd"> &gt;&gt;&gt; df = df.rename_axis(&quot;limbs&quot;, axis=&quot;columns&quot;).sort_index()</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> limbs num_legs num_arms</span>
<span class="sd"> animal</span>
<span class="sd"> cat 4 0</span>
<span class="sd"> dog 4 0</span>
<span class="sd"> monkey 2 2</span>
<span class="sd"> **MultiIndex**</span>
<span class="sd"> &gt;&gt;&gt; index = pd.MultiIndex.from_product([[&#39;mammal&#39;],</span>
<span class="sd"> ... [&#39;dog&#39;, &#39;cat&#39;, &#39;monkey&#39;]],</span>
<span class="sd"> ... names=[&#39;type&#39;, &#39;name&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&quot;num_legs&quot;: [4, 4, 2],</span>
<span class="sd"> ... &quot;num_arms&quot;: [0, 0, 2]},</span>
<span class="sd"> ... index=index,</span>
<span class="sd"> ... columns=[&quot;num_legs&quot;, &quot;num_arms&quot;])</span>
<span class="sd"> &gt;&gt;&gt; df # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs num_arms</span>
<span class="sd"> type name</span>
<span class="sd"> mammal dog 4 0</span>
<span class="sd"> cat 4 0</span>
<span class="sd"> monkey 2 2</span>
<span class="sd"> &gt;&gt;&gt; df.rename_axis(index={&#39;type&#39;: &#39;class&#39;}).sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs num_arms</span>
<span class="sd"> class name</span>
<span class="sd"> mammal cat 4 0</span>
<span class="sd"> dog 4 0</span>
<span class="sd"> monkey 2 2</span>
<span class="sd"> &gt;&gt;&gt; df.rename_axis(index=str.upper).sort_index() # doctest: +NORMALIZE_WHITESPACE</span>
<span class="sd"> num_legs num_arms</span>
<span class="sd"> TYPE NAME</span>
<span class="sd"> mammal cat 4 0</span>
<span class="sd"> dog 4 0</span>
<span class="sd"> monkey 2 2</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">gen_names</span><span class="p">(</span>
<span class="n">v</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Any</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">],</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Any</span><span class="p">],</span> <span class="n">Callable</span><span class="p">[[</span><span class="n">Name</span><span class="p">],</span> <span class="n">Any</span><span class="p">]],</span>
<span class="n">curnames</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">],</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]:</span>
<span class="n">newnames</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]</span>
<span class="k">if</span> <span class="n">is_scalar</span><span class="p">(</span><span class="n">v</span><span class="p">):</span>
<span class="n">newnames</span> <span class="o">=</span> <span class="p">[</span><span class="n">cast</span><span class="p">(</span><span class="n">Name</span><span class="p">,</span> <span class="n">v</span><span class="p">)]</span>
<span class="k">elif</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">v</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">is_dict_like</span><span class="p">(</span><span class="n">v</span><span class="p">):</span>
<span class="n">newnames</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">cast</span><span class="p">(</span><span class="n">Sequence</span><span class="p">[</span><span class="n">Name</span><span class="p">],</span> <span class="n">v</span><span class="p">))</span>
<span class="k">elif</span> <span class="n">is_dict_like</span><span class="p">(</span><span class="n">v</span><span class="p">):</span>
<span class="n">v_dict</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">Dict</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="n">v</span><span class="p">)</span>
<span class="n">newnames</span> <span class="o">=</span> <span class="p">[</span><span class="n">v_dict</span><span class="p">[</span><span class="n">name</span><span class="p">]</span> <span class="k">if</span> <span class="n">name</span> <span class="ow">in</span> <span class="n">v_dict</span> <span class="k">else</span> <span class="n">name</span> <span class="k">for</span> <span class="n">name</span> <span class="ow">in</span> <span class="n">curnames</span><span class="p">]</span>
<span class="k">elif</span> <span class="nb">callable</span><span class="p">(</span><span class="n">v</span><span class="p">):</span>
<span class="n">v_callable</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">Callable</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="n">v</span><span class="p">)</span>
<span class="n">newnames</span> <span class="o">=</span> <span class="p">[</span><span class="n">v_callable</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">curnames</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;`mapper` or `index` or `columns` should be &quot;</span>
<span class="s2">&quot;either dict-like or function type.&quot;</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">newnames</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">len</span><span class="p">(</span><span class="n">curnames</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Length of new names must be </span><span class="si">{}</span><span class="s2">, got </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">curnames</span><span class="p">),</span> <span class="nb">len</span><span class="p">(</span><span class="n">newnames</span><span class="p">))</span>
<span class="p">)</span>
<span class="k">return</span> <span class="p">[</span><span class="n">name</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">name</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="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">newnames</span><span class="p">]</span>
<span class="k">if</span> <span class="n">mapper</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="p">(</span><span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Cannot specify both &#39;mapper&#39; and any of &#39;index&#39; or &#39;columns&#39;.&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">mapper</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">index</span> <span class="o">=</span> <span class="n">mapper</span>
<span class="k">elif</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">columns</span> <span class="o">=</span> <span class="n">mapper</span>
<span class="n">column_label_names</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">gen_names</span><span class="p">(</span><span class="n">columns</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="o">.</span><span class="n">names</span><span class="p">)</span>
<span class="k">if</span> <span class="n">columns</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span>
<span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_label_names</span>
<span class="p">)</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">gen_names</span><span class="p">(</span><span class="n">index</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span><span class="p">)</span> <span class="k">if</span> <span class="n">index</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="k">else</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_names</span>
<span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span>
<span class="n">index_names</span><span class="o">=</span><span class="n">index_names</span><span class="p">,</span> <span class="n">column_label_names</span><span class="o">=</span><span class="n">column_label_names</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.keys"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.keys.html#pyspark.pandas.DataFrame.keys">[docs]</a> <span class="k">def</span> <span class="nf">keys</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">Index</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return alias for columns.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Index</span>
<span class="sd"> Columns of the DataFrame.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([[1, 2], [4, 5], [7, 8]],</span>
<span class="sd"> ... index=[&#39;cobra&#39;, &#39;viper&#39;, &#39;sidewinder&#39;],</span>
<span class="sd"> ... columns=[&#39;max_speed&#39;, &#39;shield&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> max_speed shield</span>
<span class="sd"> cobra 1 2</span>
<span class="sd"> viper 4 5</span>
<span class="sd"> sidewinder 7 8</span>
<span class="sd"> &gt;&gt;&gt; df.keys()</span>
<span class="sd"> Index([&#39;max_speed&#39;, &#39;shield&#39;], dtype=&#39;object&#39;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">columns</span></div>
<div class="viewcode-block" id="DataFrame.pct_change"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.pct_change.html#pyspark.pandas.DataFrame.pct_change">[docs]</a> <span class="k">def</span> <span class="nf">pct_change</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">periods</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="mi">1</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Percentage change between the current and a prior element.</span>
<span class="sd"> .. note:: the current implementation of this API uses Spark&#39;s Window without</span>
<span class="sd"> specifying partition specification. This leads to moving all data into</span>
<span class="sd"> a single partition in a single machine and could cause serious</span>
<span class="sd"> performance degradation. Avoid this method with very large datasets.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> periods : int, default 1</span>
<span class="sd"> Periods to shift for forming percent change.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> Percentage change in French franc, Deutsche Mark, and Italian lira</span>
<span class="sd"> from 1980-01-01 to 1980-03-01.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({</span>
<span class="sd"> ... &#39;FR&#39;: [4.0405, 4.0963, 4.3149],</span>
<span class="sd"> ... &#39;GR&#39;: [1.7246, 1.7482, 1.8519],</span>
<span class="sd"> ... &#39;IT&#39;: [804.74, 810.01, 860.13]},</span>
<span class="sd"> ... index=[&#39;1980-01-01&#39;, &#39;1980-02-01&#39;, &#39;1980-03-01&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> FR GR IT</span>
<span class="sd"> 1980-01-01 4.0405 1.7246 804.74</span>
<span class="sd"> 1980-02-01 4.0963 1.7482 810.01</span>
<span class="sd"> 1980-03-01 4.3149 1.8519 860.13</span>
<span class="sd"> &gt;&gt;&gt; df.pct_change()</span>
<span class="sd"> FR GR IT</span>
<span class="sd"> 1980-01-01 NaN NaN NaN</span>
<span class="sd"> 1980-02-01 0.013810 0.013684 0.006549</span>
<span class="sd"> 1980-03-01 0.053365 0.059318 0.061876</span>
<span class="sd"> You can set periods to shift for forming percent change</span>
<span class="sd"> &gt;&gt;&gt; df.pct_change(2)</span>
<span class="sd"> FR GR IT</span>
<span class="sd"> 1980-01-01 NaN NaN NaN</span>
<span class="sd"> 1980-02-01 NaN NaN NaN</span>
<span class="sd"> 1980-03-01 0.067912 0.073814 0.06883</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">window</span> <span class="o">=</span> <span class="n">Window</span><span class="o">.</span><span class="n">orderBy</span><span class="p">(</span><span class="n">NATURAL_ORDER_COLUMN_NAME</span><span class="p">)</span><span class="o">.</span><span class="n">rowsBetween</span><span class="p">(</span><span class="o">-</span><span class="n">periods</span><span class="p">,</span> <span class="o">-</span><span class="n">periods</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">op</span><span class="p">(</span><span class="n">psser</span><span class="p">:</span> <span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">PySparkColumn</span><span class="p">:</span>
<span class="n">prev_row</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">lag</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="p">,</span> <span class="n">periods</span><span class="p">)</span><span class="o">.</span><span class="n">over</span><span class="p">(</span><span class="n">window</span><span class="p">)</span>
<span class="k">return</span> <span class="p">((</span><span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span> <span class="o">-</span> <span class="n">prev_row</span><span class="p">)</span> <span class="o">/</span> <span class="n">prev_row</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span>
<span class="n">psser</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_apply_series_op</span><span class="p">(</span><span class="n">op</span><span class="p">,</span> <span class="n">should_resolve</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<span class="c1"># TODO(SPARK-46168): axis = 1</span>
<div class="viewcode-block" id="DataFrame.idxmax"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.idxmax.html#pyspark.pandas.DataFrame.idxmax">[docs]</a> <span class="k">def</span> <span class="nf">idxmax</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return index of first occurrence of maximum over requested axis.</span>
<span class="sd"> NA/null values are excluded.</span>
<span class="sd"> .. note:: This API collect all rows with maximum value using `to_pandas()`</span>
<span class="sd"> because we suppose the number of rows with max values are usually small in general.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> axis : 0 or &#39;index&#39;</span>
<span class="sd"> Can only be set to 0 now.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.idxmax</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 2],</span>
<span class="sd"> ... &#39;b&#39;: [4.0, 2.0, 3.0, 1.0],</span>
<span class="sd"> ... &#39;c&#39;: [300, 200, 400, 200]})</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> a b c</span>
<span class="sd"> 0 1 4.0 300</span>
<span class="sd"> 1 2 2.0 200</span>
<span class="sd"> 2 3 3.0 400</span>
<span class="sd"> 3 2 1.0 200</span>
<span class="sd"> &gt;&gt;&gt; psdf.idxmax()</span>
<span class="sd"> a 2</span>
<span class="sd"> b 0</span>
<span class="sd"> c 2</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> For Multi-column Index</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 2],</span>
<span class="sd"> ... &#39;b&#39;: [4.0, 2.0, 3.0, 1.0],</span>
<span class="sd"> ... &#39;c&#39;: [300, 200, 400, 200]})</span>
<span class="sd"> &gt;&gt;&gt; psdf.columns = pd.MultiIndex.from_tuples([(&#39;a&#39;, &#39;x&#39;), (&#39;b&#39;, &#39;y&#39;), (&#39;c&#39;, &#39;z&#39;)])</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> a b c</span>
<span class="sd"> x y z</span>
<span class="sd"> 0 1 4.0 300</span>
<span class="sd"> 1 2 2.0 200</span>
<span class="sd"> 2 3 3.0 400</span>
<span class="sd"> 3 2 1.0 200</span>
<span class="sd"> &gt;&gt;&gt; psdf.idxmax()</span>
<span class="sd"> a x 2</span>
<span class="sd"> b y 0</span>
<span class="sd"> c z 2</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">max_cols</span> <span class="o">=</span> <span class="nb">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">scol</span><span class="p">:</span> <span class="n">F</span><span class="o">.</span><span class="n">max</span><span class="p">(</span><span class="n">scol</span><span class="p">),</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">)</span>
<span class="n">sdf_max</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">max_cols</span><span class="p">)</span><span class="o">.</span><span class="n">head</span><span class="p">()</span>
<span class="c1"># `sdf_max` looks like below</span>
<span class="c1"># +------+------+------+</span>
<span class="c1"># |(a, x)|(b, y)|(c, z)|</span>
<span class="c1"># +------+------+------+</span>
<span class="c1"># | 3| 4.0| 400|</span>
<span class="c1"># +------+------+------+</span>
<span class="n">conds</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">scol</span> <span class="o">==</span> <span class="n">max_val</span> <span class="k">for</span> <span class="n">scol</span><span class="p">,</span> <span class="n">max_val</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">,</span> <span class="n">sdf_max</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">cond</span> <span class="o">=</span> <span class="n">reduce</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">:</span> <span class="n">x</span> <span class="o">|</span> <span class="n">y</span><span class="p">,</span> <span class="n">conds</span><span class="p">)</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">cond</span><span class="p">))</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span><span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">from_pandas</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span><span class="o">.</span><span class="n">idxmax</span><span class="p">()))</span></div>
<span class="c1"># TODO(SPARK-46168): axis = 1</span>
<div class="viewcode-block" id="DataFrame.idxmin"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.idxmin.html#pyspark.pandas.DataFrame.idxmin">[docs]</a> <span class="k">def</span> <span class="nf">idxmin</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;Series&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return index of first occurrence of minimum over requested axis.</span>
<span class="sd"> NA/null values are excluded.</span>
<span class="sd"> .. note:: This API collect all rows with minimum value using `to_pandas()`</span>
<span class="sd"> because we suppose the number of rows with min values are usually small in general.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> axis : 0 or &#39;index&#39;</span>
<span class="sd"> Can only be set to 0 now.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.idxmin</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 2],</span>
<span class="sd"> ... &#39;b&#39;: [4.0, 2.0, 3.0, 1.0],</span>
<span class="sd"> ... &#39;c&#39;: [300, 200, 400, 200]})</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> a b c</span>
<span class="sd"> 0 1 4.0 300</span>
<span class="sd"> 1 2 2.0 200</span>
<span class="sd"> 2 3 3.0 400</span>
<span class="sd"> 3 2 1.0 200</span>
<span class="sd"> &gt;&gt;&gt; psdf.idxmin()</span>
<span class="sd"> a 0</span>
<span class="sd"> b 3</span>
<span class="sd"> c 1</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> For Multi-column Index</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 2],</span>
<span class="sd"> ... &#39;b&#39;: [4.0, 2.0, 3.0, 1.0],</span>
<span class="sd"> ... &#39;c&#39;: [300, 200, 400, 200]})</span>
<span class="sd"> &gt;&gt;&gt; psdf.columns = pd.MultiIndex.from_tuples([(&#39;a&#39;, &#39;x&#39;), (&#39;b&#39;, &#39;y&#39;), (&#39;c&#39;, &#39;z&#39;)])</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> a b c</span>
<span class="sd"> x y z</span>
<span class="sd"> 0 1 4.0 300</span>
<span class="sd"> 1 2 2.0 200</span>
<span class="sd"> 2 3 3.0 400</span>
<span class="sd"> 3 2 1.0 200</span>
<span class="sd"> &gt;&gt;&gt; psdf.idxmin()</span>
<span class="sd"> a x 0</span>
<span class="sd"> b y 3</span>
<span class="sd"> c z 1</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">min_cols</span> <span class="o">=</span> <span class="nb">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">scol</span><span class="p">:</span> <span class="n">F</span><span class="o">.</span><span class="n">min</span><span class="p">(</span><span class="n">scol</span><span class="p">),</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">)</span>
<span class="n">sdf_min</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="n">min_cols</span><span class="p">)</span><span class="o">.</span><span class="n">head</span><span class="p">()</span>
<span class="n">conds</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">scol</span> <span class="o">==</span> <span class="n">min_val</span> <span class="k">for</span> <span class="n">scol</span><span class="p">,</span> <span class="n">min_val</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">,</span> <span class="n">sdf_min</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">cond</span> <span class="o">=</span> <span class="n">reduce</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">:</span> <span class="n">x</span> <span class="o">|</span> <span class="n">y</span><span class="p">,</span> <span class="n">conds</span><span class="p">)</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">cond</span><span class="p">))</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span><span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">,</span> <span class="n">ps</span><span class="o">.</span><span class="n">from_pandas</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span><span class="o">.</span><span class="n">idxmin</span><span class="p">()))</span></div>
<div class="viewcode-block" id="DataFrame.info"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.info.html#pyspark.pandas.DataFrame.info">[docs]</a> <span class="k">def</span> <span class="nf">info</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">verbose</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">buf</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">IO</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="n">max_cols</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">int</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">show_counts</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</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;</span>
<span class="sd"> Print a concise summary of a DataFrame.</span>
<span class="sd"> This method prints information about a DataFrame including</span>
<span class="sd"> the index dtype and column dtypes, non-null values and memory usage.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> verbose : bool, optional</span>
<span class="sd"> Whether to print the full summary.</span>
<span class="sd"> buf : writable buffer, defaults to sys.stdout</span>
<span class="sd"> Where to send the output. By default the output is printed to</span>
<span class="sd"> sys.stdout. Pass a writable buffer if you need to further process</span>
<span class="sd"> the output.</span>
<span class="sd"> max_cols : int, optional</span>
<span class="sd"> When to switch from the verbose to the truncated output. If the</span>
<span class="sd"> DataFrame has more than `max_cols` columns, the truncated output</span>
<span class="sd"> is used.</span>
<span class="sd"> show_counts : bool, optional</span>
<span class="sd"> Whether to show the non-null counts.</span>
<span class="sd"> .. versionadded:: 4.0.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> None</span>
<span class="sd"> This method prints a summary of a DataFrame and returns None.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.describe: Generate descriptive statistics of DataFrame</span>
<span class="sd"> columns.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; int_values = [1, 2, 3, 4, 5]</span>
<span class="sd"> &gt;&gt;&gt; text_values = [&#39;alpha&#39;, &#39;beta&#39;, &#39;gamma&#39;, &#39;delta&#39;, &#39;epsilon&#39;]</span>
<span class="sd"> &gt;&gt;&gt; float_values = [0.0, 0.25, 0.5, 0.75, 1.0]</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame(</span>
<span class="sd"> ... {&quot;int_col&quot;: int_values, &quot;text_col&quot;: text_values, &quot;float_col&quot;: float_values},</span>
<span class="sd"> ... columns=[&#39;int_col&#39;, &#39;text_col&#39;, &#39;float_col&#39;])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> int_col text_col float_col</span>
<span class="sd"> 0 1 alpha 0.00</span>
<span class="sd"> 1 2 beta 0.25</span>
<span class="sd"> 2 3 gamma 0.50</span>
<span class="sd"> 3 4 delta 0.75</span>
<span class="sd"> 4 5 epsilon 1.00</span>
<span class="sd"> Prints information of all columns:</span>
<span class="sd"> &gt;&gt;&gt; df.info(verbose=True) # doctest: +SKIP</span>
<span class="sd"> &lt;class &#39;pyspark.pandas.frame.DataFrame&#39;&gt;</span>
<span class="sd"> Index: 5 entries, 0 to 4</span>
<span class="sd"> Data columns (total 3 columns):</span>
<span class="sd"> # Column Non-Null Count Dtype</span>
<span class="sd"> --- ------ -------------- -----</span>
<span class="sd"> 0 int_col 5 non-null int64</span>
<span class="sd"> 1 text_col 5 non-null object</span>
<span class="sd"> 2 float_col 5 non-null float64</span>
<span class="sd"> dtypes: float64(1), int64(1), object(1)</span>
<span class="sd"> Prints a summary of columns count and its dtypes but not per column</span>
<span class="sd"> information:</span>
<span class="sd"> &gt;&gt;&gt; df.info(verbose=False) # doctest: +SKIP</span>
<span class="sd"> &lt;class &#39;pyspark.pandas.frame.DataFrame&#39;&gt;</span>
<span class="sd"> Index: 5 entries, 0 to 4</span>
<span class="sd"> Columns: 3 entries, int_col to float_col</span>
<span class="sd"> dtypes: float64(1), int64(1), object(1)</span>
<span class="sd"> Pipe output of DataFrame.info to buffer instead of sys.stdout, get</span>
<span class="sd"> buffer content and writes to a text file:</span>
<span class="sd"> &gt;&gt;&gt; import io</span>
<span class="sd"> &gt;&gt;&gt; buffer = io.StringIO()</span>
<span class="sd"> &gt;&gt;&gt; df.info(buf=buffer)</span>
<span class="sd"> &gt;&gt;&gt; s = buffer.getvalue()</span>
<span class="sd"> &gt;&gt;&gt; with open(&#39;%s/info.txt&#39; % path, &quot;w&quot;,</span>
<span class="sd"> ... encoding=&quot;utf-8&quot;) as f:</span>
<span class="sd"> ... _ = f.write(s)</span>
<span class="sd"> &gt;&gt;&gt; with open(&#39;%s/info.txt&#39; % path) as f:</span>
<span class="sd"> ... f.readlines() # doctest: +SKIP</span>
<span class="sd"> [&quot;&lt;class &#39;pyspark.pandas.frame.DataFrame&#39;&gt;\\n&quot;,</span>
<span class="sd"> &#39;Index: 5 entries, 0 to 4\\n&#39;,</span>
<span class="sd"> &#39;Data columns (total 3 columns):\\n&#39;,</span>
<span class="sd"> &#39; # Column Non-Null Count Dtype \\n&#39;,</span>
<span class="sd"> &#39;--- ------ -------------- ----- \\n&#39;,</span>
<span class="sd"> &#39; 0 int_col 5 non-null int64 \\n&#39;,</span>
<span class="sd"> &#39; 1 text_col 5 non-null object \\n&#39;,</span>
<span class="sd"> &#39; 2 float_col 5 non-null float64\\n&#39;,</span>
<span class="sd"> &#39;dtypes: float64(1), int64(1), object(1)&#39;]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># To avoid pandas&#39; existing config affects pandas-on-Spark.</span>
<span class="c1"># TODO: should we have corresponding pandas-on-Spark configs?</span>
<span class="k">with</span> <span class="n">pd</span><span class="o">.</span><span class="n">option_context</span><span class="p">(</span>
<span class="s2">&quot;display.max_info_columns&quot;</span><span class="p">,</span> <span class="n">sys</span><span class="o">.</span><span class="n">maxsize</span><span class="p">,</span> <span class="s2">&quot;display.max_info_rows&quot;</span><span class="p">,</span> <span class="n">sys</span><span class="o">.</span><span class="n">maxsize</span>
<span class="p">):</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># hack to use pandas&#39; info as is.</span>
<span class="nb">object</span><span class="o">.</span><span class="fm">__setattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_data&quot;</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span>
<span class="n">count_func</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">count</span>
<span class="bp">self</span><span class="o">.</span><span class="n">count</span> <span class="o">=</span> <span class="p">(</span> <span class="c1"># type: ignore[method-assign]</span>
<span class="k">lambda</span><span class="p">:</span> <span class="n">count_func</span><span class="p">()</span><span class="o">.</span><span class="n">_to_pandas</span><span class="p">()</span> <span class="c1"># type: ignore[assignment, misc, union-attr]</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">info</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="n">verbose</span><span class="o">=</span><span class="n">verbose</span><span class="p">,</span>
<span class="n">buf</span><span class="o">=</span><span class="n">buf</span><span class="p">,</span>
<span class="n">max_cols</span><span class="o">=</span><span class="n">max_cols</span><span class="p">,</span>
<span class="n">memory_usage</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span>
<span class="n">show_counts</span><span class="o">=</span><span class="n">show_counts</span><span class="p">,</span> <span class="c1"># type: ignore</span>
<span class="p">)</span>
<span class="k">finally</span><span class="p">:</span>
<span class="k">del</span> <span class="bp">self</span><span class="o">.</span><span class="n">_data</span>
<span class="bp">self</span><span class="o">.</span><span class="n">count</span> <span class="o">=</span> <span class="n">count_func</span> <span class="c1"># type: ignore[method-assign]</span></div>
<span class="c1"># TODO: fix parameter &#39;axis&#39; and &#39;numeric_only&#39; to work same as pandas&#39;</span>
<div class="viewcode-block" id="DataFrame.quantile"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.quantile.html#pyspark.pandas.DataFrame.quantile">[docs]</a> <span class="k">def</span> <span class="nf">quantile</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">q</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">float</span><span class="p">,</span> <span class="n">Iterable</span><span class="p">[</span><span class="nb">float</span><span class="p">]]</span> <span class="o">=</span> <span class="mf">0.5</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="n">numeric_only</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span>
<span class="n">accuracy</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="mi">10000</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">DataFrameOrSeries</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return value at the given quantile.</span>
<span class="sd"> .. note:: Unlike pandas&#39;, the quantile in pandas-on-Spark is an approximated quantile</span>
<span class="sd"> based upon approximate percentile computation because computing quantile across a</span>
<span class="sd"> large dataset is extremely expensive.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> q : float or array-like, default 0.5 (50% quantile)</span>
<span class="sd"> 0 &lt;= q &lt;= 1, the quantile(s) to compute.</span>
<span class="sd"> axis : int or str, default 0 or &#39;index&#39;</span>
<span class="sd"> Can only be set to 0 now.</span>
<span class="sd"> numeric_only : bool, default False</span>
<span class="sd"> Include only `float`, `int` or `boolean` data.</span>
<span class="sd"> .. versionchanged:: 4.0.0</span>
<span class="sd"> The default value of ``numeric_only`` is now ``False``.</span>
<span class="sd"> accuracy : int, optional</span>
<span class="sd"> Default accuracy of approximation. Larger value means better accuracy.</span>
<span class="sd"> The relative error can be deduced by 1.0 / accuracy.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> Series or DataFrame</span>
<span class="sd"> If q is an array, a DataFrame will be returned where the</span>
<span class="sd"> index is q, the columns are the columns of self, and the values are the quantiles.</span>
<span class="sd"> If q is a float, a Series will be returned where the</span>
<span class="sd"> index is the columns of self and the values are the quantiles.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; psdf = ps.DataFrame({&#39;a&#39;: [1, 2, 3, 4, 5], &#39;b&#39;: [6, 7, 8, 9, 0]})</span>
<span class="sd"> &gt;&gt;&gt; psdf</span>
<span class="sd"> a b</span>
<span class="sd"> 0 1 6</span>
<span class="sd"> 1 2 7</span>
<span class="sd"> 2 3 8</span>
<span class="sd"> 3 4 9</span>
<span class="sd"> 4 5 0</span>
<span class="sd"> &gt;&gt;&gt; psdf.quantile(.5)</span>
<span class="sd"> a 3.0</span>
<span class="sd"> b 7.0</span>
<span class="sd"> Name: 0.5, dtype: float64</span>
<span class="sd"> &gt;&gt;&gt; psdf.quantile([.25, .5, .75])</span>
<span class="sd"> a b</span>
<span class="sd"> 0.25 2.0 6.0</span>
<span class="sd"> 0.50 3.0 7.0</span>
<span class="sd"> 0.75 4.0 8.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</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">accuracy</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;accuracy must be an integer; however, got [</span><span class="si">%s</span><span class="s2">]&quot;</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">accuracy</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span>
<span class="p">)</span>
<span class="n">qq</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">float</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="nb">float</span><span class="p">]]</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">q</span><span class="p">)</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">q</span><span class="p">,</span> <span class="n">Iterable</span><span class="p">)</span> <span class="k">else</span> <span class="n">q</span>
<span class="k">for</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">qq</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">qq</span><span class="p">,</span> <span class="nb">list</span><span class="p">)</span> <span class="k">else</span> <span class="p">[</span><span class="n">qq</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">v</span><span class="p">,</span> <span class="nb">float</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;q must be a float or an array of floats; however, [</span><span class="si">%s</span><span class="s2">] found.&quot;</span> <span class="o">%</span> <span class="nb">type</span><span class="p">(</span><span class="n">v</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="n">v</span> <span class="o">&lt;</span> <span class="mf">0.0</span> <span class="ow">or</span> <span class="n">v</span> <span class="o">&gt;</span> <span class="mf">1.0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;percentiles should all be in the interval [0, 1].&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">quantile</span><span class="p">(</span><span class="n">psser</span><span class="p">:</span> <span class="s2">&quot;Series&quot;</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">PySparkColumn</span><span class="p">:</span>
<span class="n">spark_type</span> <span class="o">=</span> <span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span>
<span class="n">spark_column</span> <span class="o">=</span> <span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">spark_type</span><span class="p">,</span> <span class="p">(</span><span class="n">BooleanType</span><span class="p">,</span> <span class="n">NumericType</span><span class="p">,</span> <span class="n">NullType</span><span class="p">)):</span>
<span class="k">return</span> <span class="n">F</span><span class="o">.</span><span class="n">percentile_approx</span><span class="p">(</span><span class="n">spark_column</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">DoubleType</span><span class="p">()),</span> <span class="n">qq</span><span class="p">,</span> <span class="n">accuracy</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Could not convert </span><span class="si">{}</span><span class="s2"> (</span><span class="si">{}</span><span class="s2">) to numeric&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">spark_type_to_pandas_dtype</span><span class="p">(</span><span class="n">spark_type</span><span class="p">),</span> <span class="n">spark_type</span><span class="o">.</span><span class="n">simpleString</span><span class="p">()</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">qq</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="c1"># First calculate the percentiles from all columns and map it to each `quantiles`</span>
<span class="c1"># by creating each entry as a struct. So, it becomes an array of structs as below:</span>
<span class="c1">#</span>
<span class="c1"># +-----------------------------------------+</span>
<span class="c1"># | arrays|</span>
<span class="c1"># +-----------------------------------------+</span>
<span class="c1"># |[[0.25, 2, 6], [0.5, 3, 7], [0.75, 4, 8]]|</span>
<span class="c1"># +-----------------------------------------+</span>
<span class="n">percentile_cols</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">percentile_col_names</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">column</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="p">):</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">is_numeric_or_boolean</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">keep_column</span> <span class="o">=</span> <span class="ow">not</span> <span class="n">numeric_only</span> <span class="ow">or</span> <span class="n">is_numeric_or_boolean</span>
<span class="k">if</span> <span class="n">keep_column</span><span class="p">:</span>
<span class="n">percentile_col</span> <span class="o">=</span> <span class="n">quantile</span><span class="p">(</span><span class="n">psser</span><span class="p">)</span>
<span class="n">percentile_cols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">percentile_col</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">column</span><span class="p">))</span>
<span class="n">percentile_col_names</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">column</span><span class="p">)</span>
<span class="n">column_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">percentile_cols</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">index</span><span class="o">=</span><span class="n">qq</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">percentile_cols</span><span class="p">)</span>
<span class="c1"># Here, after select percentile cols, a spark_frame looks like below:</span>
<span class="c1"># +---------+---------+</span>
<span class="c1"># | a| b|</span>
<span class="c1"># +---------+---------+</span>
<span class="c1"># |[2, 3, 4]|[6, 7, 8]|</span>
<span class="c1"># +---------+---------+</span>
<span class="n">cols_dict</span><span class="p">:</span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">]]</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">column</span> <span class="ow">in</span> <span class="n">percentile_col_names</span><span class="p">:</span>
<span class="n">cols_dict</span><span class="p">[</span><span class="n">column</span><span class="p">]</span> <span class="o">=</span> <span class="nb">list</span><span class="p">()</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">qq</span><span class="p">)):</span>
<span class="n">cols_dict</span><span class="p">[</span><span class="n">column</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">column</span><span class="p">)[</span><span class="n">i</span><span class="p">]</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">column</span><span class="p">))</span>
<span class="n">internal_index_column</span> <span class="o">=</span> <span class="n">SPARK_DEFAULT_INDEX_NAME</span>
<span class="n">cols</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">col</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="nb">zip</span><span class="p">(</span><span class="o">*</span><span class="n">cols_dict</span><span class="o">.</span><span class="n">values</span><span class="p">())):</span>
<span class="n">cols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">struct</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">qq</span><span class="p">[</span><span class="n">i</span><span class="p">])</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">internal_index_column</span><span class="p">),</span> <span class="o">*</span><span class="n">col</span><span class="p">))</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">array</span><span class="p">(</span><span class="o">*</span><span class="n">cols</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="s2">&quot;arrays&quot;</span><span class="p">))</span>
<span class="c1"># And then, explode it and manually set the index.</span>
<span class="c1"># +-----------------+---+---+</span>
<span class="c1"># |__index_level_0__| a| b|</span>
<span class="c1"># +-----------------+---+---+</span>
<span class="c1"># | 0.25| 2| 6|</span>
<span class="c1"># | 0.5| 3| 7|</span>
<span class="c1"># | 0.75| 4| 8|</span>
<span class="c1"># +-----------------+---+---+</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;arrays&quot;</span><span class="p">)))</span><span class="o">.</span><span class="n">selectExpr</span><span class="p">(</span><span class="s2">&quot;col.*&quot;</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">internal_index_column</span><span class="p">)],</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">column_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">percentile_col_names</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">internal</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reduce_for_stat_function</span><span class="p">(</span>
<span class="n">quantile</span><span class="p">,</span> <span class="n">name</span><span class="o">=</span><span class="s2">&quot;quantile&quot;</span><span class="p">,</span> <span class="n">numeric_only</span><span class="o">=</span><span class="n">numeric_only</span>
<span class="p">)</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">qq</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.query"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.query.html#pyspark.pandas.DataFrame.query">[docs]</a> <span class="k">def</span> <span class="nf">query</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">expr</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> <span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Query the columns of a DataFrame with a boolean expression.</span>
<span class="sd"> .. note:: Internal columns that starting with a &#39;__&#39; prefix are able to access, however,</span>
<span class="sd"> they are not supposed to be accessed.</span>
<span class="sd"> .. note:: This API delegates to Spark SQL so the syntax follows Spark SQL. Therefore, the</span>
<span class="sd"> pandas specific syntax such as `@` is not supported. If you want the pandas syntax,</span>
<span class="sd"> you can work around with :meth:`DataFrame.pandas_on_spark.apply_batch`, but you should</span>
<span class="sd"> be aware that `query_func` will be executed at different nodes in a distributed manner.</span>
<span class="sd"> So, for example to use `@` syntax, make sure the variable is serialized by</span>
<span class="sd"> putting it within the closure as below.</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: range(2000), &#39;B&#39;: range(2000)})</span>
<span class="sd"> &gt;&gt;&gt; def query_func(pdf):</span>
<span class="sd"> ... num = 1995</span>
<span class="sd"> ... return pdf.query(&#39;A &gt; @num&#39;)</span>
<span class="sd"> &gt;&gt;&gt; df.pandas_on_spark.apply_batch(query_func)</span>
<span class="sd"> A B</span>
<span class="sd"> 1996 1996 1996</span>
<span class="sd"> 1997 1997 1997</span>
<span class="sd"> 1998 1998 1998</span>
<span class="sd"> 1999 1999 1999</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> expr : str</span>
<span class="sd"> The query string to evaluate.</span>
<span class="sd"> You can refer to column names that contain spaces by surrounding</span>
<span class="sd"> them in backticks.</span>
<span class="sd"> For example, if one of your columns is called ``a a`` and you want</span>
<span class="sd"> to sum it with ``b``, your query should be ```a a` + b``.</span>
<span class="sd"> inplace : bool</span>
<span class="sd"> Whether the query should modify the data in place or return</span>
<span class="sd"> a modified copy.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> DataFrame resulting from the provided query expression.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: range(1, 6),</span>
<span class="sd"> ... &#39;B&#39;: range(10, 0, -2),</span>
<span class="sd"> ... &#39;C C&#39;: range(10, 5, -1)})</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C C</span>
<span class="sd"> 0 1 10 10</span>
<span class="sd"> 1 2 8 9</span>
<span class="sd"> 2 3 6 8</span>
<span class="sd"> 3 4 4 7</span>
<span class="sd"> 4 5 2 6</span>
<span class="sd"> &gt;&gt;&gt; df.query(&#39;A &gt; B&#39;)</span>
<span class="sd"> A B C C</span>
<span class="sd"> 4 5 2 6</span>
<span class="sd"> The previous expression is equivalent to</span>
<span class="sd"> &gt;&gt;&gt; df[df.A &gt; df.B]</span>
<span class="sd"> A B C C</span>
<span class="sd"> 4 5 2 6</span>
<span class="sd"> For columns with spaces in their name, you can use backtick quoting.</span>
<span class="sd"> &gt;&gt;&gt; df.query(&#39;B == `C C`&#39;)</span>
<span class="sd"> A B C C</span>
<span class="sd"> 0 1 10 10</span>
<span class="sd"> The previous expression is equivalent to</span>
<span class="sd"> &gt;&gt;&gt; df[df.B == df[&#39;C C&#39;]]</span>
<span class="sd"> A B C C</span>
<span class="sd"> 0 1 10 10</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Doesn&#39;t support for MultiIndex columns&quot;</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">expr</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;expr must be a string to be evaluated, </span><span class="si">{}</span><span class="s2"> given&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="n">data_columns</span> <span class="o">=</span> <span class="p">[</span><span class="n">label</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">]</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_spark_columns</span>
<span class="o">+</span> <span class="p">[</span>
<span class="n">scol</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col</span><span class="p">)</span>
<span class="k">for</span> <span class="n">scol</span><span class="p">,</span> <span class="n">col</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_columns</span><span class="p">,</span> <span class="n">data_columns</span><span class="p">)</span>
<span class="p">]</span>
<span class="p">)</span><span class="o">.</span><span class="n">filter</span><span class="p">(</span><span class="n">expr</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">data_columns</span><span class="o">=</span><span class="n">data_columns</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.take"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.take.html#pyspark.pandas.DataFrame.take">[docs]</a> <span class="k">def</span> <span class="nf">take</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">indices</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="nb">int</span><span class="p">],</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the elements in the given *positional* indices along an axis.</span>
<span class="sd"> This means that we are not indexing according to actual values in</span>
<span class="sd"> the index attribute of the object. We are indexing according to the</span>
<span class="sd"> actual position of the element in the object.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> indices : array-like</span>
<span class="sd"> An array of ints indicating which positions to take.</span>
<span class="sd"> axis : {0 or &#39;index&#39;, 1 or &#39;columns&#39;, None}, default 0</span>
<span class="sd"> The axis on which to select elements. ``0`` means that we are</span>
<span class="sd"> selecting rows, ``1`` means that we are selecting columns.</span>
<span class="sd"> **kwargs</span>
<span class="sd"> For compatibility with :meth:`numpy.take`. Has no effect on the</span>
<span class="sd"> output.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> taken : same type as caller</span>
<span class="sd"> An array-like containing the elements taken from the object.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.loc : Select a subset of a DataFrame by labels.</span>
<span class="sd"> DataFrame.iloc : Select a subset of a DataFrame by positions.</span>
<span class="sd"> numpy.take : Take elements from an array along an axis.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(&#39;falcon&#39;, &#39;bird&#39;, 389.0),</span>
<span class="sd"> ... (&#39;parrot&#39;, &#39;bird&#39;, 24.0),</span>
<span class="sd"> ... (&#39;lion&#39;, &#39;mammal&#39;, 80.5),</span>
<span class="sd"> ... (&#39;monkey&#39;, &#39;mammal&#39;, np.nan)],</span>
<span class="sd"> ... columns=[&#39;name&#39;, &#39;class&#39;, &#39;max_speed&#39;],</span>
<span class="sd"> ... index=[0, 2, 3, 1])</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> name class max_speed</span>
<span class="sd"> 0 falcon bird 389.0</span>
<span class="sd"> 2 parrot bird 24.0</span>
<span class="sd"> 3 lion mammal 80.5</span>
<span class="sd"> 1 monkey mammal NaN</span>
<span class="sd"> Take elements at positions 0 and 3 along the axis 0 (default).</span>
<span class="sd"> Note how the actual indices selected (0 and 1) do not correspond to</span>
<span class="sd"> our selected indices 0 and 3. That&#39;s because we are selecting the 0th</span>
<span class="sd"> and 3rd rows, not rows whose indices equal 0 and 3.</span>
<span class="sd"> &gt;&gt;&gt; df.take([0, 3]).sort_index()</span>
<span class="sd"> name class max_speed</span>
<span class="sd"> 0 falcon bird 389.0</span>
<span class="sd"> 1 monkey mammal NaN</span>
<span class="sd"> Take elements at indices 1 and 2 along the axis 1 (column selection).</span>
<span class="sd"> &gt;&gt;&gt; df.take([1, 2], axis=1)</span>
<span class="sd"> class max_speed</span>
<span class="sd"> 0 bird 389.0</span>
<span class="sd"> 2 bird 24.0</span>
<span class="sd"> 3 mammal 80.5</span>
<span class="sd"> 1 mammal NaN</span>
<span class="sd"> We may take elements using negative integers for positive indices,</span>
<span class="sd"> starting from the end of the object, just like with Python lists.</span>
<span class="sd"> &gt;&gt;&gt; df.take([-1, -2]).sort_index()</span>
<span class="sd"> name class max_speed</span>
<span class="sd"> 1 monkey mammal NaN</span>
<span class="sd"> 3 lion mammal 80.5</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">)</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">indices</span><span class="p">)</span> <span class="ow">or</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">indices</span><span class="p">,</span> <span class="p">(</span><span class="nb">dict</span><span class="p">,</span> <span class="nb">set</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;`indices` must be a list-like except dict or set&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">iloc</span><span class="p">[</span><span class="n">indices</span><span class="p">,</span> <span class="p">:])</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:,</span> <span class="n">indices</span><span class="p">])</span></div>
<div class="viewcode-block" id="DataFrame.eval"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.eval.html#pyspark.pandas.DataFrame.eval">[docs]</a> <span class="k">def</span> <span class="nf">eval</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">expr</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> <span class="n">inplace</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="n">DataFrameOrSeries</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Evaluate a string describing operations on DataFrame columns.</span>
<span class="sd"> Operates on columns only, not specific rows or elements. This allows</span>
<span class="sd"> `eval` to run arbitrary code, which can make you vulnerable to code</span>
<span class="sd"> injection if you pass user input to this function.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> expr : str</span>
<span class="sd"> The expression string to evaluate.</span>
<span class="sd"> inplace : bool, default False</span>
<span class="sd"> If the expression contains an assignment, whether to perform the</span>
<span class="sd"> operation inplace and mutate the existing DataFrame. Otherwise,</span>
<span class="sd"> a new DataFrame is returned.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> The result of the evaluation.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.query : Evaluates a boolean expression to query the columns</span>
<span class="sd"> of a frame.</span>
<span class="sd"> DataFrame.assign : Can evaluate an expression or function to create new</span>
<span class="sd"> values for a column.</span>
<span class="sd"> eval : Evaluate a Python expression as a string using various</span>
<span class="sd"> backends.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: range(1, 6), &#39;B&#39;: range(10, 0, -2)})</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 10</span>
<span class="sd"> 1 2 8</span>
<span class="sd"> 2 3 6</span>
<span class="sd"> 3 4 4</span>
<span class="sd"> 4 5 2</span>
<span class="sd"> &gt;&gt;&gt; df.eval(&#39;A + B&#39;)</span>
<span class="sd"> 0 11</span>
<span class="sd"> 1 10</span>
<span class="sd"> 2 9</span>
<span class="sd"> 3 8</span>
<span class="sd"> 4 7</span>
<span class="sd"> dtype: int64</span>
<span class="sd"> Assignment is allowed though by default the original DataFrame is not</span>
<span class="sd"> modified.</span>
<span class="sd"> &gt;&gt;&gt; df.eval(&#39;C = A + B&#39;)</span>
<span class="sd"> A B C</span>
<span class="sd"> 0 1 10 11</span>
<span class="sd"> 1 2 8 10</span>
<span class="sd"> 2 3 6 9</span>
<span class="sd"> 3 4 4 8</span>
<span class="sd"> 4 5 2 7</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1 10</span>
<span class="sd"> 1 2 8</span>
<span class="sd"> 2 3 6</span>
<span class="sd"> 3 4 4</span>
<span class="sd"> 4 5 2</span>
<span class="sd"> Use ``inplace=True`` to modify the original DataFrame.</span>
<span class="sd"> &gt;&gt;&gt; df.eval(&#39;C = A + B&#39;, inplace=True)</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B C</span>
<span class="sd"> 0 1 10 11</span>
<span class="sd"> 1 2 8 10</span>
<span class="sd"> 2 3 6 9</span>
<span class="sd"> 3 4 4 8</span>
<span class="sd"> 4 5 2 7</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">first_series</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;`eval` is not supported for multi-index columns&quot;</span><span class="p">)</span>
<span class="n">inplace</span> <span class="o">=</span> <span class="n">validate_bool_kwarg</span><span class="p">(</span><span class="n">inplace</span><span class="p">,</span> <span class="s2">&quot;inplace&quot;</span><span class="p">)</span>
<span class="n">should_return_series</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">series_name</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">should_return_scalar</span> <span class="o">=</span> <span class="kc">False</span>
<span class="c1"># Since `eval_func` doesn&#39;t have a type hint, inferring the schema is always preformed</span>
<span class="c1"># in the `apply_batch`. Hence, the variables `should_return_series`, `series_name`,</span>
<span class="c1"># and `should_return_scalar` can be updated.</span>
<span class="k">def</span> <span class="nf">eval_func</span><span class="p">(</span><span class="n">pdf</span><span class="p">):</span> <span class="c1"># type: ignore[no-untyped-def]</span>
<span class="k">nonlocal</span> <span class="n">should_return_series</span>
<span class="k">nonlocal</span> <span class="n">series_name</span>
<span class="k">nonlocal</span> <span class="n">should_return_scalar</span>
<span class="n">result_inner</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">eval</span><span class="p">(</span><span class="n">expr</span><span class="p">,</span> <span class="n">inplace</span><span class="o">=</span><span class="n">inplace</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="n">result_inner</span> <span class="o">=</span> <span class="n">pdf</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">result_inner</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">):</span>
<span class="n">should_return_series</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">series_name</span> <span class="o">=</span> <span class="n">result_inner</span><span class="o">.</span><span class="n">name</span>
<span class="n">result_inner</span> <span class="o">=</span> <span class="n">result_inner</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="k">elif</span> <span class="n">is_scalar</span><span class="p">(</span><span class="n">result_inner</span><span class="p">):</span>
<span class="n">should_return_scalar</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">result_inner</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">(</span><span class="n">result_inner</span><span class="p">)</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="k">return</span> <span class="n">result_inner</span>
<span class="n">result</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">pandas_on_spark</span><span class="o">.</span><span class="n">apply_batch</span><span class="p">(</span><span class="n">eval_func</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inplace</span><span class="p">:</span>
<span class="c1"># Here, the result is always a frame because the error is thrown during schema inference</span>
<span class="c1"># from pandas.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">result</span><span class="o">.</span><span class="n">_internal</span><span class="p">,</span> <span class="n">check_same_anchor</span><span class="o">=</span><span class="kc">False</span><span class="p">)</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="k">elif</span> <span class="n">should_return_series</span><span class="p">:</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">result</span><span class="p">)</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">series_name</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">should_return_scalar</span><span class="p">:</span>
<span class="k">return</span> <span class="n">first_series</span><span class="p">(</span><span class="n">result</span><span class="p">)[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Returns a frame</span>
<span class="k">return</span> <span class="n">result</span></div>
<div class="viewcode-block" id="DataFrame.explode"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.explode.html#pyspark.pandas.DataFrame.explode">[docs]</a> <span class="k">def</span> <span class="nf">explode</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">column</span><span class="p">:</span> <span class="n">Name</span><span class="p">,</span> <span class="n">ignore_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Transform each element of a list-like to a row, replicating index values.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> column : str or tuple</span>
<span class="sd"> Column to explode.</span>
<span class="sd"> ignore_index : bool, default False</span>
<span class="sd"> If True, the resulting index will be labeled 0, 1, …, n - 1.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> Exploded lists to rows of the subset columns;</span>
<span class="sd"> index will be duplicated for these rows.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.unstack : Pivot a level of the (necessarily hierarchical)</span>
<span class="sd"> index labels.</span>
<span class="sd"> DataFrame.melt : Unpivot a DataFrame from wide format to long format.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;A&#39;: [[1, 2, 3], [], [3, 4]], &#39;B&#39;: 1})</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> A B</span>
<span class="sd"> 0 [1, 2, 3] 1</span>
<span class="sd"> 1 [] 1</span>
<span class="sd"> 2 [3, 4] 1</span>
<span class="sd"> &gt;&gt;&gt; df.explode(&#39;A&#39;)</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1.0 1</span>
<span class="sd"> 0 2.0 1</span>
<span class="sd"> 0 3.0 1</span>
<span class="sd"> 1 NaN 1</span>
<span class="sd"> 2 3.0 1</span>
<span class="sd"> 2 4.0 1</span>
<span class="sd"> &gt;&gt;&gt; df.explode(&#39;A&#39;, ignore_index=True)</span>
<span class="sd"> A B</span>
<span class="sd"> 0 1.0 1</span>
<span class="sd"> 1 2.0 1</span>
<span class="sd"> 2 3.0 1</span>
<span class="sd"> 3 NaN 1</span>
<span class="sd"> 4 3.0 1</span>
<span class="sd"> 5 4.0 1</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">column</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;column must be a scalar&quot;</span><span class="p">)</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="p">)</span>
<span class="n">psser</span> <span class="o">=</span> <span class="n">psdf</span><span class="p">[</span><span class="n">column</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">psser</span><span class="p">,</span> <span class="n">Series</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;The column </span><span class="si">%s</span><span class="s2"> is not unique. For a multi-index, the label must be a tuple &quot;</span>
<span class="s2">&quot;with elements corresponding to each level.&quot;</span> <span class="o">%</span> <span class="n">name_like_string</span><span class="p">(</span><span class="n">column</span><span class="p">)</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">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="n">ArrayType</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span>
<span class="n">psser</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">F</span><span class="o">.</span><span class="n">explode_outer</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span><span class="p">)</span>
<span class="p">)</span>
<span class="n">data_fields</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_fields</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">idx</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="o">.</span><span class="n">index</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">_column_label</span><span class="p">)</span>
<span class="n">field</span> <span class="o">=</span> <span class="n">data_fields</span><span class="p">[</span><span class="n">idx</span><span class="p">]</span>
<span class="n">spark_type</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">ArrayType</span><span class="p">,</span> <span class="n">field</span><span class="o">.</span><span class="n">spark_type</span><span class="p">)</span><span class="o">.</span><span class="n">elementType</span>
<span class="n">dtype</span> <span class="o">=</span> <span class="n">spark_type_to_pandas_dtype</span><span class="p">(</span><span class="n">spark_type</span><span class="p">)</span>
<span class="n">data_fields</span><span class="p">[</span><span class="n">idx</span><span class="p">]</span> <span class="o">=</span> <span class="n">field</span><span class="o">.</span><span class="n">copy</span><span class="p">(</span><span class="n">dtype</span><span class="o">=</span><span class="n">dtype</span><span class="p">,</span> <span class="n">spark_type</span><span class="o">=</span><span class="n">spark_type</span><span class="p">,</span> <span class="n">nullable</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">data_fields</span><span class="o">=</span><span class="n">data_fields</span><span class="p">)</span>
<span class="n">result_df</span><span class="p">:</span> <span class="n">DataFrame</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">return</span> <span class="n">result_df</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> <span class="k">if</span> <span class="n">ignore_index</span> <span class="k">else</span> <span class="n">result_df</span></div>
<div class="viewcode-block" id="DataFrame.mode"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.mode.html#pyspark.pandas.DataFrame.mode">[docs]</a> <span class="k">def</span> <span class="nf">mode</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="n">numeric_only</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span><span class="p">,</span> <span class="n">dropna</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Get the mode(s) of each element along the selected axis.</span>
<span class="sd"> The mode of a set of values is the value that appears most often.</span>
<span class="sd"> It can be multiple values.</span>
<span class="sd"> .. versionadded:: 3.4.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> axis : {0 or &#39;index&#39;}, default 0</span>
<span class="sd"> Axis for the function to be applied on.</span>
<span class="sd"> numeric_only : bool, default False</span>
<span class="sd"> If True, only apply to numeric columns.</span>
<span class="sd"> dropna : bool, default True</span>
<span class="sd"> Don&#39;t consider counts of NaN/NaT.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> The modes of each column or row.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.mode : Return the highest frequency value in a Series.</span>
<span class="sd"> Series.value_counts : Return the counts of values in a Series.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame([(&#39;bird&#39;, 2, 2),</span>
<span class="sd"> ... (&#39;mammal&#39;, 4, np.nan),</span>
<span class="sd"> ... (&#39;arthropod&#39;, 8, 0),</span>
<span class="sd"> ... (&#39;bird&#39;, 2, np.nan)],</span>
<span class="sd"> ... index=(&#39;falcon&#39;, &#39;horse&#39;, &#39;spider&#39;, &#39;ostrich&#39;),</span>
<span class="sd"> ... columns=(&#39;species&#39;, &#39;legs&#39;, &#39;wings&#39;))</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> species legs wings</span>
<span class="sd"> falcon bird 2 2.0</span>
<span class="sd"> horse mammal 4 NaN</span>
<span class="sd"> spider arthropod 8 0.0</span>
<span class="sd"> ostrich bird 2 NaN</span>
<span class="sd"> By default missing values are not considered, and the mode of wings</span>
<span class="sd"> are both 0 and 2. Because the resulting DataFrame has two rows,</span>
<span class="sd"> the second row of ``species`` and ``legs`` contains ``NaN``.</span>
<span class="sd"> &gt;&gt;&gt; df.mode()</span>
<span class="sd"> species legs wings</span>
<span class="sd"> 0 bird 2.0 0.0</span>
<span class="sd"> 1 None NaN 2.0</span>
<span class="sd"> Setting ``dropna=False`` ``NaN`` values are considered and they can be</span>
<span class="sd"> the mode (like for wings).</span>
<span class="sd"> &gt;&gt;&gt; df.mode(dropna=False)</span>
<span class="sd"> species legs wings</span>
<span class="sd"> 0 bird 2 NaN</span>
<span class="sd"> Setting ``numeric_only=True``, only the mode of numeric columns is</span>
<span class="sd"> computed, and columns of other types are ignored.</span>
<span class="sd"> &gt;&gt;&gt; df.mode(numeric_only=True)</span>
<span class="sd"> legs wings</span>
<span class="sd"> 0 2.0 0.0</span>
<span class="sd"> 1 NaN 2.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">,</span> <span class="n">none_axis</span><span class="o">=</span><span class="mi">0</span><span class="p">)</span>
<span class="k">if</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;axis should be either 0 or &quot;index&quot; currently.&#39;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">numeric_only</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">numeric_only</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">mode_scols</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">mode_col_names</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">mode_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">label</span><span class="p">,</span> <span class="n">col_name</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">data_spark_column_names</span>
<span class="p">):</span>
<span class="n">psser</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="n">is_numeric</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">data_type</span><span class="p">,</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">))</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">numeric_only</span> <span class="ow">or</span> <span class="n">is_numeric</span><span class="p">:</span>
<span class="n">scol</span> <span class="o">=</span> <span class="n">psser</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">column</span>
<span class="n">mode_scol</span> <span class="o">=</span> <span class="n">SF</span><span class="o">.</span><span class="n">mode</span><span class="p">(</span><span class="n">scol</span><span class="p">,</span> <span class="n">dropna</span><span class="p">)</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">col_name</span><span class="p">)</span>
<span class="n">mode_scols</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">mode_scol</span><span class="p">)</span>
<span class="n">mode_col_names</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">col_name</span><span class="p">)</span>
<span class="n">mode_labels</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">label</span><span class="p">)</span>
<span class="c1"># Here, after aggregation, a spark_frame looks like below:</span>
<span class="c1"># +-------+----+----------+</span>
<span class="c1"># |species|legs| wings|</span>
<span class="c1"># +-------+----+----------+</span>
<span class="c1"># | [bird]| [2]|[0.0, 2.0]|</span>
<span class="c1"># +-------+----+----------+</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">mode_scols</span><span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="o">*</span><span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">array_sort</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">name</span><span class="p">))</span><span class="o">.</span><span class="n">alias</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">mode_col_names</span><span class="p">])</span>
<span class="n">zip_col_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;__mode_zip_tmp_col__&quot;</span><span class="p">)</span>
<span class="n">explode_col_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="s2">&quot;__mode_explode_tmp_col__&quot;</span><span class="p">)</span>
<span class="c1"># After this transformation, sdf turns out to be:</span>
<span class="c1"># +-------+----+-----+</span>
<span class="c1"># |species|legs|wings|</span>
<span class="c1"># +-------+----+-----+</span>
<span class="c1"># | bird| 2| 0.0|</span>
<span class="c1"># | NULL|NULL| 2.0|</span>
<span class="c1"># +-------+----+-----+</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">sdf</span><span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">arrays_zip</span><span class="p">(</span><span class="o">*</span><span class="p">[</span><span class="n">F</span><span class="o">.</span><span class="n">col</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">mode_col_names</span><span class="p">])</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">zip_col_name</span><span class="p">))</span>
<span class="o">.</span><span class="n">select</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">explode</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="n">zip_col_name</span><span class="p">))</span><span class="o">.</span><span class="n">alias</span><span class="p">(</span><span class="n">explode_col_name</span><span class="p">))</span>
<span class="o">.</span><span class="n">select</span><span class="p">(</span>
<span class="o">*</span><span class="p">[</span>
<span class="n">F</span><span class="o">.</span><span class="n">col</span><span class="p">(</span><span class="s2">&quot;</span><span class="si">{0}</span><span class="s2">.</span><span class="si">{1}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">explode_col_name</span><span class="p">,</span> <span class="n">name</span><span class="p">))</span><span class="o">.</span><span class="n">alias</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">mode_col_names</span>
<span class="p">]</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">withColumn</span><span class="p">(</span><span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">,</span> <span class="n">F</span><span class="o">.</span><span class="n">monotonically_increasing_id</span><span class="p">())</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">InternalFrame</span><span class="p">(</span>
<span class="n">spark_frame</span><span class="o">=</span><span class="n">sdf</span><span class="p">,</span>
<span class="n">index_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">SPARK_DEFAULT_INDEX_NAME</span><span class="p">)],</span>
<span class="n">column_labels</span><span class="o">=</span><span class="n">mode_labels</span><span class="p">,</span>
<span class="n">data_spark_columns</span><span class="o">=</span><span class="p">[</span><span class="n">scol_for</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">col</span><span class="p">)</span> <span class="k">for</span> <span class="n">col</span> <span class="ow">in</span> <span class="n">mode_col_names</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">internal</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.tail"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.tail.html#pyspark.pandas.DataFrame.tail">[docs]</a> <span class="k">def</span> <span class="nf">tail</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="mi">5</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the last `n` rows.</span>
<span class="sd"> This function returns last `n` rows from the object based on</span>
<span class="sd"> position. It is useful for quickly verifying data, for example,</span>
<span class="sd"> after sorting or appending rows.</span>
<span class="sd"> For negative values of `n`, this function returns all rows except</span>
<span class="sd"> the first `n` rows, equivalent to ``df[n:]``.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> n : int, default 5</span>
<span class="sd"> Number of rows to select.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> type of caller</span>
<span class="sd"> The last `n` rows of the caller object.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.head : The first `n` rows of the caller object.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = ps.DataFrame({&#39;animal&#39;: [&#39;alligator&#39;, &#39;bee&#39;, &#39;falcon&#39;, &#39;lion&#39;,</span>
<span class="sd"> ... &#39;monkey&#39;, &#39;parrot&#39;, &#39;shark&#39;, &#39;whale&#39;, &#39;zebra&#39;]})</span>
<span class="sd"> &gt;&gt;&gt; df</span>
<span class="sd"> animal</span>
<span class="sd"> 0 alligator</span>
<span class="sd"> 1 bee</span>
<span class="sd"> 2 falcon</span>
<span class="sd"> 3 lion</span>
<span class="sd"> 4 monkey</span>
<span class="sd"> 5 parrot</span>
<span class="sd"> 6 shark</span>
<span class="sd"> 7 whale</span>
<span class="sd"> 8 zebra</span>
<span class="sd"> Viewing the last 5 lines</span>
<span class="sd"> &gt;&gt;&gt; df.tail() # doctest: +SKIP</span>
<span class="sd"> animal</span>
<span class="sd"> 4 monkey</span>
<span class="sd"> 5 parrot</span>
<span class="sd"> 6 shark</span>
<span class="sd"> 7 whale</span>
<span class="sd"> 8 zebra</span>
<span class="sd"> Viewing the last `n` lines (three in this case)</span>
<span class="sd"> &gt;&gt;&gt; df.tail(3) # doctest: +SKIP</span>
<span class="sd"> animal</span>
<span class="sd"> 6 shark</span>
<span class="sd"> 7 whale</span>
<span class="sd"> 8 zebra</span>
<span class="sd"> For negative values of `n`</span>
<span class="sd"> &gt;&gt;&gt; df.tail(-3) # doctest: +SKIP</span>
<span class="sd"> animal</span>
<span class="sd"> 3 lion</span>
<span class="sd"> 4 monkey</span>
<span class="sd"> 5 parrot</span>
<span class="sd"> 6 shark</span>
<span class="sd"> 7 whale</span>
<span class="sd"> 8 zebra</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">n</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;bad operand type for unary -: &#39;</span><span class="si">{}</span><span class="s2">&#39;&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">n</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">))</span>
<span class="k">if</span> <span class="n">n</span> <span class="o">&lt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">n</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">+</span> <span class="n">n</span>
<span class="k">if</span> <span class="n">n</span> <span class="o">&lt;=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">ps</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_filter</span><span class="p">(</span><span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">False</span><span class="p">)))</span>
<span class="c1"># Should use `resolved_copy` here for the case like `(psdf + 1).tail()`</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="n">rows</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">tail</span><span class="p">(</span><span class="n">n</span><span class="p">)</span>
<span class="n">new_sdf</span> <span class="o">=</span> <span class="n">default_session</span><span class="p">()</span><span class="o">.</span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">rows</span><span class="p">,</span> <span class="n">sdf</span><span class="o">.</span><span class="n">schema</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_sdf</span><span class="p">(</span><span class="n">new_sdf</span><span class="p">))</span></div>
<div class="viewcode-block" id="DataFrame.align"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.align.html#pyspark.pandas.DataFrame.align">[docs]</a> <span class="k">def</span> <span class="nf">align</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">other</span><span class="p">:</span> <span class="n">DataFrameOrSeries</span><span class="p">,</span>
<span class="n">join</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;outer&quot;</span><span class="p">,</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Axis</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">copy</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Tuple</span><span class="p">[</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">,</span> <span class="n">DataFrameOrSeries</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Align two objects on their axes with the specified join method.</span>
<span class="sd"> Join method is specified for each axis Index.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> other : DataFrame or Series</span>
<span class="sd"> join : {{&#39;outer&#39;, &#39;inner&#39;, &#39;left&#39;, &#39;right&#39;}}, default &#39;outer&#39;</span>
<span class="sd"> axis : allowed axis of the other object, default None</span>
<span class="sd"> Align on index (0), columns (1), or both (None).</span>
<span class="sd"> copy : bool, default True</span>
<span class="sd"> Always returns new objects. If copy=False and no reindexing is</span>
<span class="sd"> required then original objects are returned.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> (left, right) : (DataFrame, type of other)</span>
<span class="sd"> Aligned objects.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; ps.set_option(&quot;compute.ops_on_diff_frames&quot;, True)</span>
<span class="sd"> &gt;&gt;&gt; df1 = ps.DataFrame({&quot;a&quot;: [1, 2, 3], &quot;b&quot;: [&quot;a&quot;, &quot;b&quot;, &quot;c&quot;]}, index=[10, 20, 30])</span>
<span class="sd"> &gt;&gt;&gt; df2 = ps.DataFrame({&quot;a&quot;: [4, 5, 6], &quot;c&quot;: [&quot;d&quot;, &quot;e&quot;, &quot;f&quot;]}, index=[10, 11, 12])</span>
<span class="sd"> Align both axis:</span>
<span class="sd"> &gt;&gt;&gt; aligned_l, aligned_r = df1.align(df2)</span>
<span class="sd"> &gt;&gt;&gt; aligned_l.sort_index()</span>
<span class="sd"> a b c</span>
<span class="sd"> 10 1.0 a NaN</span>
<span class="sd"> 11 NaN None NaN</span>
<span class="sd"> 12 NaN None NaN</span>
<span class="sd"> 20 2.0 b NaN</span>
<span class="sd"> 30 3.0 c NaN</span>
<span class="sd"> &gt;&gt;&gt; aligned_r.sort_index()</span>
<span class="sd"> a b c</span>
<span class="sd"> 10 4.0 NaN d</span>
<span class="sd"> 11 5.0 NaN e</span>
<span class="sd"> 12 6.0 NaN f</span>
<span class="sd"> 20 NaN NaN None</span>
<span class="sd"> 30 NaN NaN None</span>
<span class="sd"> Align only axis=0 (index):</span>
<span class="sd"> &gt;&gt;&gt; aligned_l, aligned_r = df1.align(df2, axis=0)</span>
<span class="sd"> &gt;&gt;&gt; aligned_l.sort_index()</span>
<span class="sd"> a b</span>
<span class="sd"> 10 1.0 a</span>
<span class="sd"> 11 NaN None</span>
<span class="sd"> 12 NaN None</span>
<span class="sd"> 20 2.0 b</span>
<span class="sd"> 30 3.0 c</span>
<span class="sd"> &gt;&gt;&gt; aligned_r.sort_index()</span>
<span class="sd"> a c</span>
<span class="sd"> 10 4.0 d</span>
<span class="sd"> 11 5.0 e</span>
<span class="sd"> 12 6.0 f</span>
<span class="sd"> 20 NaN None</span>
<span class="sd"> 30 NaN None</span>
<span class="sd"> Align only axis=1 (column):</span>
<span class="sd"> &gt;&gt;&gt; aligned_l, aligned_r = df1.align(df2, axis=1)</span>
<span class="sd"> &gt;&gt;&gt; aligned_l.sort_index()</span>
<span class="sd"> a b c</span>
<span class="sd"> 10 1 a NaN</span>
<span class="sd"> 20 2 b NaN</span>
<span class="sd"> 30 3 c NaN</span>
<span class="sd"> &gt;&gt;&gt; aligned_r.sort_index()</span>
<span class="sd"> a b c</span>
<span class="sd"> 10 4 NaN d</span>
<span class="sd"> 11 5 NaN e</span>
<span class="sd"> 12 6 NaN f</span>
<span class="sd"> Align with the join type &quot;inner&quot;:</span>
<span class="sd"> &gt;&gt;&gt; aligned_l, aligned_r = df1.align(df2, join=&quot;inner&quot;)</span>
<span class="sd"> &gt;&gt;&gt; aligned_l.sort_index()</span>
<span class="sd"> a</span>
<span class="sd"> 10 1</span>
<span class="sd"> &gt;&gt;&gt; aligned_r.sort_index()</span>
<span class="sd"> a</span>
<span class="sd"> 10 4</span>
<span class="sd"> Align with a Series:</span>
<span class="sd"> &gt;&gt;&gt; s = ps.Series([7, 8, 9], index=[10, 11, 12])</span>
<span class="sd"> &gt;&gt;&gt; aligned_l, aligned_r = df1.align(s, axis=0)</span>
<span class="sd"> &gt;&gt;&gt; aligned_l.sort_index()</span>
<span class="sd"> a b</span>
<span class="sd"> 10 1.0 a</span>
<span class="sd"> 11 NaN None</span>
<span class="sd"> 12 NaN None</span>
<span class="sd"> 20 2.0 b</span>
<span class="sd"> 30 3.0 c</span>
<span class="sd"> &gt;&gt;&gt; aligned_r.sort_index()</span>
<span class="sd"> 10 7.0</span>
<span class="sd"> 11 8.0</span>
<span class="sd"> 12 9.0</span>
<span class="sd"> 20 NaN</span>
<span class="sd"> 30 NaN</span>
<span class="sd"> dtype: float64</span>
<span class="sd"> &gt;&gt;&gt; ps.reset_option(&quot;compute.ops_on_diff_frames&quot;)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span><span class="p">,</span> <span class="n">first_series</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="p">(</span><span class="n">DataFrame</span><span class="p">,</span> <span class="n">Series</span><span class="p">)):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;unsupported type: </span><span class="si">{}</span><span class="s2">&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">other</span><span class="p">)</span><span class="o">.</span><span class="vm">__name__</span><span class="p">))</span>
<span class="n">how</span> <span class="o">=</span> <span class="n">validate_how</span><span class="p">(</span><span class="n">join</span><span class="p">)</span>
<span class="n">axis</span> <span class="o">=</span> <span class="n">validate_axis</span><span class="p">(</span><span class="n">axis</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="n">right_is_series</span> <span class="o">=</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">Series</span><span class="p">)</span>
<span class="k">if</span> <span class="n">right_is_series</span><span class="p">:</span>
<span class="k">if</span> <span class="n">axis</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Must specify axis=0 or 1&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">axis</span> <span class="o">!=</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span>
<span class="s2">&quot;align currently only works for axis=0 when right is Series&quot;</span>
<span class="p">)</span>
<span class="n">left</span> <span class="o">=</span> <span class="bp">self</span>
<span class="n">right</span> <span class="o">=</span> <span class="n">other</span>
<span class="k">if</span> <span class="p">(</span><span class="n">axis</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">0</span><span class="p">)</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">same_anchor</span><span class="p">(</span><span class="n">left</span><span class="p">,</span> <span class="n">right</span><span class="p">):</span>
<span class="n">combined</span> <span class="o">=</span> <span class="n">combine_frames</span><span class="p">(</span><span class="n">left</span><span class="p">,</span> <span class="n">right</span><span class="p">,</span> <span class="n">how</span><span class="o">=</span><span class="n">how</span><span class="p">)</span>
<span class="n">left</span> <span class="o">=</span> <span class="n">combined</span><span class="p">[</span><span class="s2">&quot;this&quot;</span><span class="p">]</span>
<span class="n">right</span> <span class="o">=</span> <span class="n">combined</span><span class="p">[</span><span class="s2">&quot;that&quot;</span><span class="p">]</span>
<span class="k">if</span> <span class="n">right_is_series</span><span class="p">:</span>
<span class="n">right</span> <span class="o">=</span> <span class="n">first_series</span><span class="p">(</span><span class="n">cast</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">[</span><span class="n">Any</span><span class="p">],</span> <span class="n">right</span><span class="p">))</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">name</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span>
<span class="n">axis</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="n">axis</span> <span class="o">==</span> <span class="mi">1</span>
<span class="p">)</span> <span class="ow">and</span> <span class="n">left</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span> <span class="o">!=</span> <span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">left</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">!=</span> <span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;cannot join with no overlapping index names&quot;</span><span class="p">)</span>
<span class="n">left</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">right</span> <span class="o">=</span> <span class="n">right</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="k">if</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;full&quot;</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="nb">sorted</span><span class="p">(</span>
<span class="nb">list</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">left</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">|</span> <span class="nb">set</span><span class="p">(</span><span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">))</span>
<span class="p">)</span>
<span class="k">elif</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;inner&quot;</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="nb">sorted</span><span class="p">(</span>
<span class="nb">list</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">left</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">)</span> <span class="o">&amp;</span> <span class="nb">set</span><span class="p">(</span><span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">))</span>
<span class="p">)</span>
<span class="k">elif</span> <span class="n">how</span> <span class="o">==</span> <span class="s2">&quot;left&quot;</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="n">left</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">column_labels</span> <span class="o">=</span> <span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">left</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">left</span><span class="p">[</span><span class="n">label</span><span class="p">]</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">DoubleType</span><span class="p">())</span>
<span class="n">left</span> <span class="o">=</span> <span class="n">left</span><span class="p">[</span><span class="n">column_labels</span><span class="p">]</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="n">label</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">right</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">right</span><span class="p">[</span><span class="n">label</span><span class="p">]</span> <span class="o">=</span> <span class="n">F</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">DoubleType</span><span class="p">())</span>
<span class="n">right</span> <span class="o">=</span> <span class="n">right</span><span class="p">[</span><span class="n">column_labels</span><span class="p">]</span>
<span class="k">return</span> <span class="p">(</span><span class="n">left</span><span class="o">.</span><span class="n">copy</span><span class="p">(),</span> <span class="n">right</span><span class="o">.</span><span class="n">copy</span><span class="p">())</span> <span class="k">if</span> <span class="n">copy</span> <span class="k">else</span> <span class="p">(</span><span class="n">left</span><span class="p">,</span> <span class="n">right</span><span class="p">)</span></div>
<div class="viewcode-block" id="DataFrame.from_dict"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.from_dict.html#pyspark.pandas.DataFrame.from_dict">[docs]</a> <span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">from_dict</span><span class="p">(</span>
<span class="n">data</span><span class="p">:</span> <span class="n">Dict</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Any</span><span class="p">]],</span>
<span class="n">orient</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">&quot;columns&quot;</span><span class="p">,</span>
<span class="n">dtype</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="n">Dtype</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="n">columns</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="n">Name</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Construct DataFrame from dict of array-like or dicts.</span>
<span class="sd"> Creates DataFrame object from dictionary by columns or by index</span>
<span class="sd"> allowing dtype specification.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> data : dict</span>
<span class="sd"> Of the form {field : array-like} or {field : dict}.</span>
<span class="sd"> orient : {&#39;columns&#39;, &#39;index&#39;}, default &#39;columns&#39;</span>
<span class="sd"> The &quot;orientation&quot; of the data. If the keys of the passed dict</span>
<span class="sd"> should be the columns of the resulting DataFrame, pass &#39;columns&#39;</span>
<span class="sd"> (default). Otherwise, if the keys should be rows, pass &#39;index&#39;.</span>
<span class="sd"> dtype : dtype, default None</span>
<span class="sd"> Data type to force, otherwise infer.</span>
<span class="sd"> columns : list, default None</span>
<span class="sd"> Column labels to use when ``orient=&#39;index&#39;``. Raises a ValueError</span>
<span class="sd"> if used with ``orient=&#39;columns&#39;``.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrame</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> DataFrame.from_records : DataFrame from structured ndarray, sequence</span>
<span class="sd"> of tuples or dicts, or DataFrame.</span>
<span class="sd"> DataFrame : DataFrame object creation using constructor.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> By default the keys of the dict become the DataFrame columns:</span>
<span class="sd"> &gt;&gt;&gt; data = {&#39;col_1&#39;: [3, 2, 1, 0], &#39;col_2&#39;: [10, 20, 30, 40]}</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame.from_dict(data)</span>
<span class="sd"> col_1 col_2</span>
<span class="sd"> 0 3 10</span>
<span class="sd"> 1 2 20</span>
<span class="sd"> 2 1 30</span>
<span class="sd"> 3 0 40</span>
<span class="sd"> Specify ``orient=&#39;index&#39;`` to create the DataFrame using dictionary</span>
<span class="sd"> keys as rows:</span>
<span class="sd"> &gt;&gt;&gt; data = {&#39;row_1&#39;: [3, 2, 1, 0], &#39;row_2&#39;: [10, 20, 30, 40]}</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame.from_dict(data, orient=&#39;index&#39;).sort_index()</span>
<span class="sd"> 0 1 2 3</span>
<span class="sd"> row_1 3 2 1 0</span>
<span class="sd"> row_2 10 20 30 40</span>
<span class="sd"> When using the &#39;index&#39; orientation, the column names can be</span>
<span class="sd"> specified manually:</span>
<span class="sd"> &gt;&gt;&gt; ps.DataFrame.from_dict(data, orient=&#39;index&#39;,</span>
<span class="sd"> ... columns=[&#39;A&#39;, &#39;B&#39;, &#39;C&#39;, &#39;D&#39;]).sort_index()</span>
<span class="sd"> A B C D</span>
<span class="sd"> row_1 3 2 1 0</span>
<span class="sd"> row_2 10 20 30 40</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span>
<span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="o">.</span><span class="n">from_dict</span><span class="p">(</span>
<span class="n">data</span><span class="p">,</span> <span class="n">orient</span><span class="o">=</span><span class="n">orient</span><span class="p">,</span> <span class="n">dtype</span><span class="o">=</span><span class="n">dtype</span><span class="p">,</span> <span class="n">columns</span><span class="o">=</span><span class="n">columns</span> <span class="c1"># type: ignore[arg-type]</span>
<span class="p">)</span>
<span class="p">)</span></div>
<span class="c1"># Override the `groupby` to specify the actual return type annotation.</span>
<div class="viewcode-block" id="DataFrame.groupby"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.groupby.html#pyspark.pandas.DataFrame.groupby">[docs]</a> <span class="k">def</span> <span class="nf">groupby</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">by</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="s2">&quot;Series&quot;</span><span class="p">,</span> <span class="n">List</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="s2">&quot;Series&quot;</span><span class="p">]]],</span>
<span class="n">axis</span><span class="p">:</span> <span class="n">Axis</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span>
<span class="n">as_index</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">dropna</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">True</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrameGroupBy&quot;</span><span class="p">:</span>
<span class="k">return</span> <span class="n">cast</span><span class="p">(</span>
<span class="s2">&quot;DataFrameGroupBy&quot;</span><span class="p">,</span> <span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">by</span><span class="o">=</span><span class="n">by</span><span class="p">,</span> <span class="n">axis</span><span class="o">=</span><span class="n">axis</span><span class="p">,</span> <span class="n">as_index</span><span class="o">=</span><span class="n">as_index</span><span class="p">,</span> <span class="n">dropna</span><span class="o">=</span><span class="n">dropna</span><span class="p">)</span>
<span class="p">)</span></div>
<span class="n">groupby</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">Frame</span><span class="o">.</span><span class="n">groupby</span><span class="o">.</span><span class="vm">__doc__</span>
<span class="k">def</span> <span class="nf">_build_groupby</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">by</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Union</span><span class="p">[</span><span class="s2">&quot;Series&quot;</span><span class="p">,</span> <span class="n">Label</span><span class="p">]],</span> <span class="n">as_index</span><span class="p">:</span> <span class="nb">bool</span><span class="p">,</span> <span class="n">dropna</span><span class="p">:</span> <span class="nb">bool</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrameGroupBy&quot;</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.groupby</span> <span class="kn">import</span> <span class="n">DataFrameGroupBy</span>
<span class="k">return</span> <span class="n">DataFrameGroupBy</span><span class="o">.</span><span class="n">_build</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">by</span><span class="p">,</span> <span class="n">as_index</span><span class="o">=</span><span class="n">as_index</span><span class="p">,</span> <span class="n">dropna</span><span class="o">=</span><span class="n">dropna</span><span class="p">)</span>
<div class="viewcode-block" id="DataFrame.resample"><a class="viewcode-back" href="../../../reference/pyspark.pandas/api/pyspark.pandas.DataFrame.resample.html#pyspark.pandas.DataFrame.resample">[docs]</a> <span class="k">def</span> <span class="nf">resample</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span>
<span class="n">rule</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span>
<span class="n">closed</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="n">label</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="n">on</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="s2">&quot;Series&quot;</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrameResampler&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Resample time-series data.</span>
<span class="sd"> Convenience method for frequency conversion and resampling of time series.</span>
<span class="sd"> The object must have a datetime-like index (only support `DatetimeIndex` for now),</span>
<span class="sd"> or the caller must pass the label of a datetime-like</span>
<span class="sd"> series/index to the ``on`` keyword parameter.</span>
<span class="sd"> .. versionadded:: 3.4.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> rule : str</span>
<span class="sd"> The offset string or object representing target conversion.</span>
<span class="sd"> Currently, supported units are {&#39;Y&#39;, &#39;A&#39;, &#39;M&#39;, &#39;D&#39;, &#39;H&#39;,</span>
<span class="sd"> &#39;T&#39;, &#39;MIN&#39;, &#39;S&#39;}.</span>
<span class="sd"> closed : {{&#39;right&#39;, &#39;left&#39;}}, default None</span>
<span class="sd"> Which side of bin interval is closed. The default is &#39;left&#39;</span>
<span class="sd"> for all frequency offsets except for &#39;A&#39;, &#39;Y&#39; and &#39;M&#39; which all</span>
<span class="sd"> have a default of &#39;right&#39;.</span>
<span class="sd"> label : {{&#39;right&#39;, &#39;left&#39;}}, default None</span>
<span class="sd"> Which bin edge label to label bucket with. The default is &#39;left&#39;</span>
<span class="sd"> for all frequency offsets except for &#39;A&#39;, &#39;Y&#39; and &#39;M&#39; which all</span>
<span class="sd"> have a default of &#39;right&#39;.</span>
<span class="sd"> on : Series, optional</span>
<span class="sd"> For a DataFrame, column to use instead of index for resampling.</span>
<span class="sd"> Column must be datetime-like.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> DataFrameResampler</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> Series.resample : Resample a Series.</span>
<span class="sd"> groupby : Group by mapping, function, label, or list of labels.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.indexes</span> <span class="kn">import</span> <span class="n">DatetimeIndex</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.resample</span> <span class="kn">import</span> <span class="n">DataFrameResampler</span>
<span class="k">if</span> <span class="n">on</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="p">,</span> <span class="n">DatetimeIndex</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;resample currently works only for DatetimeIndex&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">on</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span>
<span class="n">as_spark_type</span><span class="p">(</span><span class="n">on</span><span class="o">.</span><span class="n">dtype</span><span class="p">),</span> <span class="p">(</span><span class="n">TimestampType</span><span class="p">,</span> <span class="n">TimestampNTZType</span><span class="p">)</span>
<span class="p">):</span>
<span class="k">raise</span> <span class="ne">NotImplementedError</span><span class="p">(</span><span class="s2">&quot;`on` currently works only for TimestampType&quot;</span><span class="p">)</span>
<span class="n">agg_columns</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">ps</span><span class="o">.</span><span class="n">Series</span><span class="p">]</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">column_label</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_type_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">),</span> <span class="p">(</span><span class="n">NumericType</span><span class="p">,</span> <span class="n">BooleanType</span><span class="p">)):</span>
<span class="n">agg_columns</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">column_label</span><span class="p">))</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">agg_columns</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;No available aggregation columns!&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrameResampler</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">=</span><span class="bp">self</span><span class="p">,</span>
<span class="n">resamplekey</span><span class="o">=</span><span class="n">on</span><span class="p">,</span>
<span class="n">rule</span><span class="o">=</span><span class="n">rule</span><span class="p">,</span>
<span class="n">closed</span><span class="o">=</span><span class="n">closed</span><span class="p">,</span>
<span class="n">label</span><span class="o">=</span><span class="n">label</span><span class="p">,</span>
<span class="n">agg_columns</span><span class="o">=</span><span class="n">agg_columns</span><span class="p">,</span>
<span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_to_internal_pandas</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a pandas DataFrame directly from _internal to avoid overhead of copy.</span>
<span class="sd"> This method is for internal use only.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">to_pandas_frame</span>
<span class="k">def</span> <span class="nf">_get_or_create_repr_pandas_cache</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">n</span><span class="p">:</span> <span class="nb">int</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Union</span><span class="p">[</span><span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">,</span> <span class="n">pd</span><span class="o">.</span><span class="n">Series</span><span class="p">]:</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_repr_pandas_cache&quot;</span><span class="p">)</span> <span class="ow">or</span> <span class="n">n</span> <span class="ow">not</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_repr_pandas_cache</span><span class="p">:</span>
<span class="nb">object</span><span class="o">.</span><span class="fm">__setattr__</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_repr_pandas_cache&quot;</span><span class="p">,</span> <span class="p">{</span><span class="n">n</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">head</span><span class="p">(</span><span class="n">n</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()}</span>
<span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_repr_pandas_cache</span><span class="p">[</span><span class="n">n</span><span class="p">]</span>
<span class="k">def</span> <span class="fm">__repr__</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">str</span><span class="p">:</span>
<span class="n">max_display_count</span> <span class="o">=</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;display.max_rows&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">max_display_count</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span><span class="o">.</span><span class="n">to_string</span><span class="p">()</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_or_create_repr_pandas_cache</span><span class="p">(</span><span class="n">max_display_count</span><span class="p">))</span>
<span class="n">pdf_length</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="s2">&quot;DataFrame&quot;</span><span class="p">,</span> <span class="n">pdf</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:</span><span class="n">max_display_count</span><span class="p">])</span>
<span class="k">if</span> <span class="n">pdf_length</span> <span class="o">&gt;</span> <span class="n">max_display_count</span><span class="p">:</span>
<span class="n">repr_string</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">to_string</span><span class="p">(</span><span class="n">show_dimensions</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">match</span> <span class="o">=</span> <span class="n">REPR_PATTERN</span><span class="o">.</span><span class="n">search</span><span class="p">(</span><span class="n">repr_string</span><span class="p">)</span>
<span class="k">if</span> <span class="n">match</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">nrows</span> <span class="o">=</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="s2">&quot;rows&quot;</span><span class="p">)</span>
<span class="n">ncols</span> <span class="o">=</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="s2">&quot;columns&quot;</span><span class="p">)</span>
<span class="n">footer</span> <span class="o">=</span> <span class="s2">&quot;</span><span class="se">\n\n</span><span class="s2">[Showing only the first </span><span class="si">{nrows}</span><span class="s2"> rows x </span><span class="si">{ncols}</span><span class="s2"> columns]&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">nrows</span><span class="o">=</span><span class="n">nrows</span><span class="p">,</span> <span class="n">ncols</span><span class="o">=</span><span class="n">ncols</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">REPR_PATTERN</span><span class="o">.</span><span class="n">sub</span><span class="p">(</span><span class="n">footer</span><span class="p">,</span> <span class="n">repr_string</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pdf</span><span class="o">.</span><span class="n">to_string</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">_repr_html_</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">str</span><span class="p">:</span>
<span class="n">max_display_count</span> <span class="o">=</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;display.max_rows&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">max_display_count</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_to_internal_pandas</span><span class="p">()</span><span class="o">.</span><span class="n">to_html</span><span class="p">(</span><span class="n">notebook</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_get_or_create_repr_pandas_cache</span><span class="p">(</span><span class="n">max_display_count</span><span class="p">)</span>
<span class="n">pdf_length</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">pdf</span><span class="p">)</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">iloc</span><span class="p">[:</span><span class="n">max_display_count</span><span class="p">]</span>
<span class="k">if</span> <span class="n">pdf_length</span> <span class="o">&gt;</span> <span class="n">max_display_count</span><span class="p">:</span>
<span class="n">repr_html</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">to_html</span><span class="p">(</span><span class="n">show_dimensions</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">notebook</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">match</span> <span class="o">=</span> <span class="n">REPR_HTML_PATTERN</span><span class="o">.</span><span class="n">search</span><span class="p">(</span><span class="n">repr_html</span><span class="p">)</span>
<span class="k">if</span> <span class="n">match</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">nrows</span> <span class="o">=</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="s2">&quot;rows&quot;</span><span class="p">)</span>
<span class="n">ncols</span> <span class="o">=</span> <span class="n">match</span><span class="o">.</span><span class="n">group</span><span class="p">(</span><span class="s2">&quot;columns&quot;</span><span class="p">)</span>
<span class="n">by</span> <span class="o">=</span> <span class="nb">chr</span><span class="p">(</span><span class="mi">215</span><span class="p">)</span>
<span class="n">footer</span> <span class="o">=</span> <span class="p">(</span>
<span class="s2">&quot;</span><span class="se">\n</span><span class="s2">&lt;p&gt;Showing only the first </span><span class="si">{rows}</span><span class="s2"> rows &quot;</span>
<span class="s2">&quot;</span><span class="si">{by}</span><span class="s2"> </span><span class="si">{cols}</span><span class="s2"> columns&lt;/p&gt;</span><span class="se">\n</span><span class="s2">&lt;/div&gt;&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">rows</span><span class="o">=</span><span class="n">nrows</span><span class="p">,</span> <span class="n">by</span><span class="o">=</span><span class="n">by</span><span class="p">,</span> <span class="n">cols</span><span class="o">=</span><span class="n">ncols</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">REPR_HTML_PATTERN</span><span class="o">.</span><span class="n">sub</span><span class="p">(</span><span class="n">footer</span><span class="p">,</span> <span class="n">repr_html</span><span class="p">)</span>
<span class="k">return</span> <span class="n">pdf</span><span class="o">.</span><span class="n">to_html</span><span class="p">(</span><span class="n">notebook</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__getitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Any</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="k">if</span> <span class="n">key</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span><span class="s2">&quot;none key&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">Series</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">key</span><span class="o">.</span><span class="n">astype</span><span class="p">(</span><span class="nb">bool</span><span class="p">)]</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">slice</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">n</span><span class="p">)</span> <span class="o">==</span> <span class="nb">int</span> <span class="ow">or</span> <span class="kc">None</span> <span class="k">for</span> <span class="n">n</span> <span class="ow">in</span> <span class="p">[</span><span class="n">key</span><span class="o">.</span><span class="n">start</span><span class="p">,</span> <span class="n">key</span><span class="o">.</span><span class="n">stop</span><span class="p">]):</span>
<span class="c1"># Seems like pandas Frame always uses int as positional search when slicing</span>
<span class="c1"># with ints.</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">iloc</span><span class="p">[</span><span class="n">key</span><span class="p">]</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[</span><span class="n">key</span><span class="p">]</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">key</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[:,</span> <span class="n">key</span><span class="p">]</span>
<span class="k">elif</span> <span class="n">is_list_like</span><span class="p">(</span><span class="n">key</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[:,</span> <span class="nb">list</span><span class="p">(</span><span class="n">key</span><span class="p">)]</span>
<span class="k">def</span> <span class="fm">__setitem__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="n">value</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="p">(</span><span class="n">DataFrame</span><span class="p">,</span> <span class="n">Series</span><span class="p">))</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">same_anchor</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="bp">self</span><span class="p">):</span>
<span class="c1"># Different Series or DataFrames</span>
<span class="n">level</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span>
<span class="n">key</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="o">.</span><span class="n">_index_normalized_label</span><span class="p">(</span><span class="n">level</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span>
<span class="n">value</span> <span class="o">=</span> <span class="n">DataFrame</span><span class="o">.</span><span class="n">_index_normalized_frame</span><span class="p">(</span><span class="n">level</span><span class="p">,</span> <span class="n">value</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">assign_columns</span><span class="p">(</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span><span class="p">,</span> <span class="n">this_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">],</span> <span class="n">that_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterator</span><span class="p">[</span><span class="n">Tuple</span><span class="p">[</span><span class="s2">&quot;Series&quot;</span><span class="p">,</span> <span class="n">Label</span><span class="p">]]:</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">key</span><span class="p">)</span> <span class="o">==</span> <span class="nb">len</span><span class="p">(</span><span class="n">that_column_labels</span><span class="p">)</span>
<span class="c1"># Note that here intentionally uses `zip_longest` that combine</span>
<span class="c1"># that_columns.</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">this_label</span><span class="p">,</span> <span class="n">that_label</span> <span class="ow">in</span> <span class="n">zip_longest</span><span class="p">(</span>
<span class="n">key</span><span class="p">,</span> <span class="n">this_column_labels</span><span class="p">,</span> <span class="n">that_column_labels</span>
<span class="p">):</span>
<span class="k">yield</span> <span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">that_label</span><span class="p">),</span> <span class="nb">tuple</span><span class="p">([</span><span class="s2">&quot;that&quot;</span><span class="p">,</span> <span class="o">*</span><span class="n">k</span><span class="p">]))</span>
<span class="k">if</span> <span class="n">this_label</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">this_label</span><span class="p">[</span><span class="mi">1</span><span class="p">:]</span> <span class="o">!=</span> <span class="n">k</span><span class="p">:</span>
<span class="k">yield</span> <span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">this_label</span><span class="p">),</span> <span class="n">this_label</span><span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">align_diff_frames</span><span class="p">(</span><span class="n">assign_columns</span><span class="p">,</span> <span class="bp">self</span><span class="p">,</span> <span class="n">value</span><span class="p">,</span> <span class="n">fillna</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">how</span><span class="o">=</span><span class="s2">&quot;left&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">len</span><span class="p">(</span><span class="n">value</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Length of values does not match length of index&quot;</span><span class="p">)</span>
<span class="c1"># TODO: avoid using default index?</span>
<span class="k">with</span> <span class="n">option_context</span><span class="p">(</span>
<span class="s2">&quot;compute.default_index_type&quot;</span><span class="p">,</span>
<span class="s2">&quot;distributed-sequence&quot;</span><span class="p">,</span>
<span class="s2">&quot;compute.ops_on_diff_frames&quot;</span><span class="p">,</span>
<span class="kc">True</span><span class="p">,</span>
<span class="p">):</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">reset_index</span><span class="p">()</span>
<span class="n">psdf</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</span> <span class="n">ps</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">(</span><span class="n">value</span><span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">psdf</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">columns</span><span class="p">[:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">index_level</span><span class="p">])</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="nb">list</span><span class="p">):</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">value</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">)</span>
<span class="c1"># Same DataFrames.</span>
<span class="n">field_names</span> <span class="o">=</span> <span class="n">value</span><span class="o">.</span><span class="n">columns</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_assign</span><span class="p">({</span><span class="n">k</span><span class="p">:</span> <span class="n">value</span><span class="p">[</span><span class="n">c</span><span class="p">]</span> <span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">c</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">key</span><span class="p">,</span> <span class="n">field_names</span><span class="p">)})</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># Same Series.</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_assign</span><span class="p">({</span><span class="n">key</span><span class="p">:</span> <span class="n">value</span><span class="p">})</span>
<span class="c1"># Since Spark 3.4, df.__setitem__ generates a new dataframe instead of operating</span>
<span class="c1"># in-place to follow pandas v1.4 behavior, see also SPARK-38946.</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_update_internal_frame</span><span class="p">(</span><span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="p">,</span> <span class="n">anchor_force_disconnect</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_index_normalized_label</span><span class="p">(</span><span class="n">level</span><span class="p">:</span> <span class="nb">int</span><span class="p">,</span> <span class="n">labels</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Name</span><span class="p">,</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">Name</span><span class="p">]])</span> <span class="o">-&gt;</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns a label that is normalized against the current column index level.</span>
<span class="sd"> For example, the key &quot;abc&quot; can be (&quot;abc&quot;, &quot;&quot;, &quot;&quot;) if the current Frame has</span>
<span class="sd"> a multi-index for its column</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">labels</span><span class="p">):</span>
<span class="n">labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">labels</span><span class="p">]</span>
<span class="k">elif</span> <span class="n">is_name_like_value</span><span class="p">(</span><span class="n">labels</span><span class="p">):</span>
<span class="n">labels</span> <span class="o">=</span> <span class="p">[(</span><span class="n">labels</span><span class="p">,)]</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">labels</span> <span class="o">=</span> <span class="p">[</span><span class="n">k</span> <span class="k">if</span> <span class="n">is_name_like_tuple</span><span class="p">(</span><span class="n">k</span><span class="p">)</span> <span class="k">else</span> <span class="p">(</span><span class="n">k</span><span class="p">,)</span> <span class="k">for</span> <span class="n">k</span> <span class="ow">in</span> <span class="n">labels</span><span class="p">]</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">&gt;</span> <span class="n">level</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">labels</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">KeyError</span><span class="p">(</span>
<span class="s2">&quot;Key length (</span><span class="si">{}</span><span class="s2">) exceeds index depth (</span><span class="si">{}</span><span class="s2">)&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="nb">max</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">labels</span><span class="p">),</span> <span class="n">level</span>
<span class="p">)</span>
<span class="p">)</span>
<span class="k">return</span> <span class="p">[</span><span class="nb">tuple</span><span class="p">(</span><span class="nb">list</span><span class="p">(</span><span class="n">label</span><span class="p">)</span> <span class="o">+</span> <span class="p">([</span><span class="s2">&quot;&quot;</span><span class="p">]</span> <span class="o">*</span> <span class="p">(</span><span class="n">level</span> <span class="o">-</span> <span class="nb">len</span><span class="p">(</span><span class="n">label</span><span class="p">))))</span> <span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">labels</span><span class="p">]</span>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_index_normalized_frame</span><span class="p">(</span><span class="n">level</span><span class="p">:</span> <span class="nb">int</span><span class="p">,</span> <span class="n">psser_or_psdf</span><span class="p">:</span> <span class="n">DataFrameOrSeries</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns a frame that is normalized against the current column index level.</span>
<span class="sd"> For example, the name in `pd.Series([...], name=&quot;abc&quot;)` can be can be</span>
<span class="sd"> (&quot;abc&quot;, &quot;&quot;, &quot;&quot;) if the current DataFrame has a multi-index for its column</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.pandas.series</span> <span class="kn">import</span> <span class="n">Series</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">psser_or_psdf</span><span class="p">,</span> <span class="n">Series</span><span class="p">):</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">psser_or_psdf</span><span class="o">.</span><span class="n">to_frame</span><span class="p">()</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">psser_or_psdf</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">),</span> <span class="nb">type</span><span class="p">(</span><span class="n">psser_or_psdf</span><span class="p">)</span>
<span class="n">psdf</span> <span class="o">=</span> <span class="n">psser_or_psdf</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">columns</span> <span class="o">=</span> <span class="n">pd</span><span class="o">.</span><span class="n">MultiIndex</span><span class="o">.</span><span class="n">from_tuples</span><span class="p">(</span>
<span class="p">[</span>
<span class="nb">tuple</span><span class="p">([</span><span class="n">name_like_string</span><span class="p">(</span><span class="n">label</span><span class="p">)]</span> <span class="o">+</span> <span class="p">([</span><span class="s2">&quot;&quot;</span><span class="p">]</span> <span class="o">*</span> <span class="p">(</span><span class="n">level</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)))</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span>
<span class="p">],</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">psdf</span>
<span class="k">def</span> <span class="nf">_build_fallback_method</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">method</span><span class="p">:</span> <span class="nb">str</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Callable</span><span class="p">:</span>
<span class="k">def</span> <span class="nf">_internal_fallback_function</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="n">log_advice</span><span class="p">(</span>
<span class="sa">f</span><span class="s2">&quot;`</span><span class="si">{</span><span class="n">method</span><span class="si">}</span><span class="s2">` is executed in fallback mode. It loads partial data into the &quot;</span>
<span class="sa">f</span><span class="s2">&quot;driver&#39;s memory to infer the schema, and loads all data into one executor&#39;s &quot;</span>
<span class="sa">f</span><span class="s2">&quot;memory to compute. It should only be used if the pandas DataFrame is expected &quot;</span>
<span class="sa">f</span><span class="s2">&quot;to be small.&quot;</span>
<span class="p">)</span>
<span class="n">input_df</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="n">index_names</span> <span class="o">=</span> <span class="n">input_df</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span>
<span class="n">sdf</span> <span class="o">=</span> <span class="n">input_df</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">spark_frame</span>
<span class="n">tmp_agg_column_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span>
<span class="n">sdf</span><span class="p">,</span> <span class="sa">f</span><span class="s2">&quot;__tmp_aggregate_col_for_frame_</span><span class="si">{</span><span class="n">method</span><span class="si">}</span><span class="s2">__&quot;</span>
<span class="p">)</span>
<span class="n">input_df</span><span class="p">[</span><span class="n">tmp_agg_column_name</span><span class="p">]</span> <span class="o">=</span> <span class="mi">0</span>
<span class="n">tmp_idx_column_name</span> <span class="o">=</span> <span class="n">verify_temp_column_name</span><span class="p">(</span>
<span class="n">sdf</span><span class="p">,</span> <span class="sa">f</span><span class="s2">&quot;__tmp_index_col_for_frame_</span><span class="si">{</span><span class="n">method</span><span class="si">}</span><span class="s2">__&quot;</span>
<span class="p">)</span>
<span class="n">input_df</span><span class="p">[</span><span class="n">tmp_idx_column_name</span><span class="p">]</span> <span class="o">=</span> <span class="n">input_df</span><span class="o">.</span><span class="n">index</span>
<span class="c1"># TODO(SPARK-46859): specify the return type if possible</span>
<span class="k">def</span> <span class="nf">compute_function</span><span class="p">(</span><span class="n">pdf</span><span class="p">:</span> <span class="n">pd</span><span class="o">.</span><span class="n">DataFrame</span><span class="p">):</span> <span class="c1"># type: ignore[no-untyped-def]</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">drop</span><span class="p">(</span><span class="n">columns</span><span class="o">=</span><span class="p">[</span><span class="n">tmp_agg_column_name</span><span class="p">])</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">tmp_idx_column_name</span><span class="p">,</span> <span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">sort_index</span><span class="p">()</span>
<span class="n">pdf</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">pdf</span><span class="p">,</span> <span class="n">method</span><span class="p">)(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="n">pdf</span><span class="p">[</span><span class="n">tmp_idx_column_name</span><span class="p">]</span> <span class="o">=</span> <span class="n">pdf</span><span class="o">.</span><span class="n">index</span>
<span class="k">return</span> <span class="n">pdf</span><span class="o">.</span><span class="n">reset_index</span><span class="p">(</span><span class="n">drop</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">output_df</span> <span class="o">=</span> <span class="n">input_df</span><span class="o">.</span><span class="n">groupby</span><span class="p">(</span><span class="n">tmp_agg_column_name</span><span class="p">)</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">compute_function</span><span class="p">)</span>
<span class="n">output_df</span> <span class="o">=</span> <span class="n">output_df</span><span class="o">.</span><span class="n">set_index</span><span class="p">(</span><span class="n">tmp_idx_column_name</span><span class="p">)</span>
<span class="n">output_df</span><span class="o">.</span><span class="n">index</span><span class="o">.</span><span class="n">names</span> <span class="o">=</span> <span class="n">index_names</span>
<span class="k">return</span> <span class="n">output_df</span>
<span class="k">return</span> <span class="n">_internal_fallback_function</span>
<span class="k">def</span> <span class="nf">_asfreq_fallback</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="n">_f</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_build_fallback_method</span><span class="p">(</span><span class="s2">&quot;asfreq&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_asof_fallback</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="n">_f</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_build_fallback_method</span><span class="p">(</span><span class="s2">&quot;asof&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_convert_dtypes_fallback</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="n">_f</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_build_fallback_method</span><span class="p">(</span><span class="s2">&quot;convert_dtypes&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_infer_objects_fallback</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="n">_f</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_build_fallback_method</span><span class="p">(</span><span class="s2">&quot;infer_objects&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_set_axis_fallback</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="n">_f</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_build_fallback_method</span><span class="p">(</span><span class="s2">&quot;set_axis&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_to_feather_fallback</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">_f</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_build_fallback_driver_method</span><span class="p">(</span><span class="s2">&quot;to_feather&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_to_stata_fallback</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">_f</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_build_fallback_driver_method</span><span class="p">(</span><span class="s2">&quot;to_stata&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_f</span><span class="p">(</span><span class="o">*</span><span class="n">args</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__getattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">:</span> <span class="nb">str</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Any</span><span class="p">:</span>
<span class="k">if</span> <span class="n">key</span><span class="o">.</span><span class="n">startswith</span><span class="p">(</span><span class="s2">&quot;__&quot;</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">AttributeError</span><span class="p">(</span><span class="n">key</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="n">MissingPandasLikeDataFrame</span><span class="p">,</span> <span class="n">key</span><span class="p">):</span>
<span class="k">if</span> <span class="n">get_option</span><span class="p">(</span><span class="s2">&quot;compute.pandas_fallback&quot;</span><span class="p">):</span>
<span class="n">new_key</span> <span class="o">=</span> <span class="sa">f</span><span class="s2">&quot;_</span><span class="si">{</span><span class="n">key</span><span class="si">}</span><span class="s2">_fallback&quot;</span>
<span class="k">if</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">new_key</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">new_key</span><span class="p">)</span>
<span class="n">property_or_func</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">MissingPandasLikeDataFrame</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">property_or_func</span><span class="p">,</span> <span class="nb">property</span><span class="p">):</span>
<span class="k">return</span> <span class="n">property_or_func</span><span class="o">.</span><span class="n">fget</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">partial</span><span class="p">(</span><span class="n">property_or_func</span><span class="p">,</span> <span class="bp">self</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">loc</span><span class="p">[:,</span> <span class="n">key</span><span class="p">]</span>
<span class="k">except</span> <span class="ne">KeyError</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">AttributeError</span><span class="p">(</span>
<span class="s2">&quot;&#39;</span><span class="si">%s</span><span class="s2">&#39; object has no attribute &#39;</span><span class="si">%s</span><span class="s2">&#39;&quot;</span> <span class="o">%</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="vm">__class__</span><span class="o">.</span><span class="vm">__name__</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span>
<span class="p">)</span>
<span class="k">def</span> <span class="fm">__setattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> <span class="n">value</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="nb">object</span><span class="o">.</span><span class="fm">__getattribute__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">object</span><span class="o">.</span><span class="fm">__setattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">key</span><span class="p">,</span> <span class="n">value</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">AttributeError</span><span class="p">:</span>
<span class="k">pass</span>
<span class="k">if</span> <span class="p">(</span><span class="n">key</span><span class="p">,)</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="bp">self</span><span class="p">[</span><span class="n">key</span><span class="p">]</span> <span class="o">=</span> <span class="n">value</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">msg</span> <span class="o">=</span> <span class="s2">&quot;pandas-on-Spark doesn&#39;t allow columns to be created via a new attribute name&quot;</span>
<span class="k">if</span> <span class="n">is_testing</span><span class="p">():</span>
<span class="k">raise</span> <span class="ne">AssertionError</span><span class="p">(</span><span class="n">msg</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span><span class="n">msg</span><span class="p">,</span> <span class="ne">UserWarning</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__len__</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">int</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">count</span><span class="p">()</span>
<span class="k">def</span> <span class="fm">__dir__</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterable</span><span class="p">[</span><span class="nb">str</span><span class="p">]:</span>
<span class="n">fields</span> <span class="o">=</span> <span class="p">[</span>
<span class="n">f</span> <span class="k">for</span> <span class="n">f</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">resolved_copy</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">schema</span><span class="o">.</span><span class="n">fieldNames</span><span class="p">()</span> <span class="k">if</span> <span class="s2">&quot; &quot;</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">f</span>
<span class="p">]</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="nb">super</span><span class="p">()</span><span class="o">.</span><span class="fm">__dir__</span><span class="p">())</span> <span class="o">+</span> <span class="n">fields</span>
<span class="k">def</span> <span class="fm">__iter__</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterator</span><span class="p">[</span><span class="n">Name</span><span class="p">]:</span>
<span class="k">return</span> <span class="nb">iter</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">columns</span><span class="p">)</span>
<span class="c1"># NDArray Compat</span>
<span class="k">def</span> <span class="nf">__array_ufunc__</span><span class="p">(</span>
<span class="bp">self</span><span class="p">,</span> <span class="n">ufunc</span><span class="p">:</span> <span class="n">Callable</span><span class="p">,</span> <span class="n">method</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> <span class="o">*</span><span class="n">inputs</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="n">Any</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="s2">&quot;DataFrame&quot;</span><span class="p">:</span>
<span class="c1"># TODO: is it possible to deduplicate it with &#39;_map_series_op&#39;?</span>
<span class="k">if</span> <span class="nb">all</span><span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">inp</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">)</span> <span class="k">for</span> <span class="n">inp</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">)</span> <span class="ow">and</span> <span class="nb">any</span><span class="p">(</span>
<span class="ow">not</span> <span class="n">same_anchor</span><span class="p">(</span><span class="n">inp</span><span class="p">,</span> <span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> <span class="k">for</span> <span class="n">inp</span> <span class="ow">in</span> <span class="n">inputs</span>
<span class="p">):</span>
<span class="c1"># binary only</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">inputs</span><span class="p">)</span> <span class="o">==</span> <span class="mi">2</span>
<span class="n">this</span> <span class="o">=</span> <span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="n">that</span> <span class="o">=</span> <span class="n">inputs</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span>
<span class="k">if</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span> <span class="o">!=</span> <span class="n">that</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels_level</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;cannot join with no overlapping index names&quot;</span><span class="p">)</span>
<span class="c1"># Different DataFrames</span>
<span class="k">def</span> <span class="nf">apply_op</span><span class="p">(</span>
<span class="n">psdf</span><span class="p">:</span> <span class="n">DataFrame</span><span class="p">,</span> <span class="n">this_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">],</span> <span class="n">that_column_labels</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">Label</span><span class="p">]</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Iterator</span><span class="p">[</span><span class="n">Tuple</span><span class="p">[</span><span class="s2">&quot;Series&quot;</span><span class="p">,</span> <span class="n">Label</span><span class="p">]]:</span>
<span class="k">for</span> <span class="n">this_label</span><span class="p">,</span> <span class="n">that_label</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">this_column_labels</span><span class="p">,</span> <span class="n">that_column_labels</span><span class="p">):</span>
<span class="k">yield</span> <span class="p">(</span>
<span class="n">ufunc</span><span class="p">(</span>
<span class="n">psdf</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">this_label</span><span class="p">),</span> <span class="n">psdf</span><span class="o">.</span><span class="n">_psser_for</span><span class="p">(</span><span class="n">that_label</span><span class="p">),</span> <span class="o">**</span><span class="n">kwargs</span>
<span class="p">)</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">this_label</span><span class="p">),</span>
<span class="n">this_label</span><span class="p">,</span>
<span class="p">)</span>
<span class="k">return</span> <span class="n">align_diff_frames</span><span class="p">(</span><span class="n">apply_op</span><span class="p">,</span> <span class="n">this</span><span class="p">,</span> <span class="n">that</span><span class="p">,</span> <span class="n">fillna</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">how</span><span class="o">=</span><span class="s2">&quot;full&quot;</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># DataFrame and Series</span>
<span class="n">applied</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">this</span> <span class="o">=</span> <span class="n">inputs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">assert</span> <span class="nb">all</span><span class="p">(</span><span class="n">inp</span> <span class="ow">is</span> <span class="n">this</span> <span class="k">for</span> <span class="n">inp</span> <span class="ow">in</span> <span class="n">inputs</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">inp</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">))</span>
<span class="k">for</span> <span class="n">label</span> <span class="ow">in</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">column_labels</span><span class="p">:</span>
<span class="n">arguments</span> <span class="o">=</span> <span class="p">[]</span>
<span class="k">for</span> <span class="n">inp</span> <span class="ow">in</span> <span class="n">inputs</span><span class="p">:</span>
<span class="n">arguments</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">inp</span><span class="p">[</span><span class="n">label</span><span class="p">]</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">inp</span><span class="p">,</span> <span class="n">DataFrame</span><span class="p">)</span> <span class="k">else</span> <span class="n">inp</span><span class="p">)</span>
<span class="c1"># both binary and unary.</span>
<span class="n">applied</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">ufunc</span><span class="p">(</span><span class="o">*</span><span class="n">arguments</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">)</span><span class="o">.</span><span class="n">rename</span><span class="p">(</span><span class="n">label</span><span class="p">))</span>
<span class="n">internal</span> <span class="o">=</span> <span class="n">this</span><span class="o">.</span><span class="n">_internal</span><span class="o">.</span><span class="n">with_new_columns</span><span class="p">(</span><span class="n">applied</span><span class="p">)</span>
<span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">internal</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">__class_getitem__</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">params</span><span class="p">:</span> <span class="n">Any</span><span class="p">)</span> <span class="o">-&gt;</span> <span class="nb">object</span><span class="p">:</span>
<span class="c1"># See https://github.com/python/typing/issues/193</span>
<span class="c1"># we always wraps the given type hints by a tuple to mimic the variadic generic.</span>
<span class="k">return</span> <span class="n">create_tuple_for_frame_type</span><span class="p">(</span><span class="n">params</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_reduce_spark_multi</span><span class="p">(</span><span class="n">sdf</span><span class="p">:</span> <span class="n">PySparkDataFrame</span><span class="p">,</span> <span class="n">aggs</span><span class="p">:</span> <span class="n">List</span><span class="p">[</span><span class="n">PySparkColumn</span><span class="p">])</span> <span class="o">-&gt;</span> <span class="n">Any</span><span class="p">:</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Performs a reduction on a spark DataFrame, the functions being known SQL aggregate functions.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">SparkDataFrame</span> <span class="o">=</span> <span class="n">get_dataframe_class</span><span class="p">()</span>
<span class="k">assert</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">SparkDataFrame</span><span class="p">)</span>
<span class="n">sdf0</span> <span class="o">=</span> <span class="n">sdf</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="o">*</span><span class="n">aggs</span><span class="p">)</span>
<span class="n">lst</span> <span class="o">=</span> <span class="n">sdf0</span><span class="o">.</span><span class="n">limit</span><span class="p">(</span><span class="mi">2</span><span class="p">)</span><span class="o">.</span><span class="n">toPandas</span><span class="p">()</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">lst</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span><span class="p">,</span> <span class="p">(</span><span class="n">sdf</span><span class="p">,</span> <span class="n">lst</span><span class="p">)</span>
<span class="n">row</span> <span class="o">=</span> <span class="n">lst</span><span class="o">.</span><span class="n">iloc</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="n">lst2</span> <span class="o">=</span> <span class="nb">list</span><span class="p">(</span><span class="n">row</span><span class="p">)</span>
<span class="k">assert</span> <span class="nb">len</span><span class="p">(</span><span class="n">lst2</span><span class="p">)</span> <span class="o">==</span> <span class="nb">len</span><span class="p">(</span><span class="n">aggs</span><span class="p">),</span> <span class="p">(</span><span class="n">row</span><span class="p">,</span> <span class="n">lst2</span><span class="p">)</span>
<span class="k">return</span> <span class="n">lst2</span>
<span class="k">class</span> <span class="nc">CachedDataFrame</span><span class="p">(</span><span class="n">DataFrame</span><span class="p">):</span>
<span class="w"> </span><span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Cached pandas-on-Spark DataFrame, which corresponds to pandas DataFrame logically, but</span>
<span class="sd"> internally it caches the corresponding Spark DataFrame.</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">internal</span><span class="p">:</span> <span class="n">InternalFrame</span><span class="p">,</span> <span class="n">storage_level</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">StorageLevel</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">):</span>
<span class="k">if</span> <span class="n">storage_level</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="nb">object</span><span class="o">.</span><span class="fm">__setattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_cached&quot;</span><span class="p">,</span> <span class="n">internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">cache</span><span class="p">())</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">storage_level</span><span class="p">,</span> <span class="n">StorageLevel</span><span class="p">):</span>
<span class="nb">object</span><span class="o">.</span><span class="fm">__setattr__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="s2">&quot;_cached&quot;</span><span class="p">,</span> <span class="n">internal</span><span class="o">.</span><span class="n">spark_frame</span><span class="o">.</span><span class="n">persist</span><span class="p">(</span><span class="n">storage_level</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Only a valid pyspark.StorageLevel type is acceptable for the `storage_level`&quot;</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">internal</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="o">-&gt;</span> <span class="s2">&quot;CachedDataFrame&quot;</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="n">exception_type</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">Type</span><span class="p">[</span><span class="ne">BaseException</span><span class="p">]],</span>
<span class="n">exception_value</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="ne">BaseException</span><span class="p">],</span>
<span class="n">traceback</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">TracebackType</span><span class="p">],</span>
<span class="p">)</span> <span class="o">-&gt;</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">bool</span><span class="p">]:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">unpersist</span><span class="p">()</span>
<span class="k">return</span> <span class="kc">None</span>
<span class="c1"># create accessor for Spark related methods.</span>
<span class="n">spark</span> <span class="o">=</span> <span class="n">CachedAccessor</span><span class="p">(</span><span class="s2">&quot;spark&quot;</span><span class="p">,</span> <span class="n">CachedSparkFrameMethods</span><span class="p">)</span>
<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">os</span>
<span class="kn">import</span> <span class="nn">doctest</span>
<span class="kn">import</span> <span class="nn">shutil</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">import</span> <span class="nn">tempfile</span>
<span class="kn">import</span> <span class="nn">uuid</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.pandas.frame</span>
<span class="n">os</span><span class="o">.</span><span class="n">chdir</span><span class="p">(</span><span class="n">os</span><span class="o">.</span><span class="n">environ</span><span class="p">[</span><span class="s2">&quot;SPARK_HOME&quot;</span><span class="p">])</span>
<span class="n">globs</span> <span class="o">=</span> <span class="n">pyspark</span><span class="o">.</span><span class="n">pandas</span><span class="o">.</span><span class="n">frame</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">globs</span><span class="p">[</span><span class="s2">&quot;ps&quot;</span><span class="p">]</span> <span class="o">=</span> <span class="n">pyspark</span><span class="o">.</span><span class="n">pandas</span>
<span class="n">spark</span> <span class="o">=</span> <span class="p">(</span>
<span class="n">SparkSession</span><span class="o">.</span><span class="n">builder</span><span class="o">.</span><span class="n">master</span><span class="p">(</span><span class="s2">&quot;local[4]&quot;</span><span class="p">)</span><span class="o">.</span><span class="n">appName</span><span class="p">(</span><span class="s2">&quot;pyspark.pandas.frame tests&quot;</span><span class="p">)</span><span class="o">.</span><span class="n">getOrCreate</span><span class="p">()</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">spark</span>
<span class="n">db_name</span> <span class="o">=</span> <span class="s2">&quot;db</span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="nb">str</span><span class="p">(</span><span class="n">uuid</span><span class="o">.</span><span class="n">uuid4</span><span class="p">())</span><span class="o">.</span><span class="n">replace</span><span class="p">(</span><span class="s2">&quot;-&quot;</span><span class="p">,</span> <span class="s2">&quot;&quot;</span><span class="p">)</span>
<span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">&quot;CREATE DATABASE </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="n">db_name</span><span class="p">)</span>
<span class="n">globs</span><span class="p">[</span><span class="s2">&quot;db&quot;</span><span class="p">]</span> <span class="o">=</span> <span class="n">db_name</span>
<span class="n">path</span> <span class="o">=</span> <span class="n">tempfile</span><span class="o">.</span><span class="n">mkdtemp</span><span class="p">()</span>
<span class="n">globs</span><span class="p">[</span><span class="s2">&quot;path&quot;</span><span class="p">]</span> <span class="o">=</span> <span class="n">path</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">pandas</span><span class="o">.</span><span class="n">frame</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">optionflags</span><span class="o">=</span><span class="n">doctest</span><span class="o">.</span><span class="n">ELLIPSIS</span> <span class="o">|</span> <span class="n">doctest</span><span class="o">.</span><span class="n">NORMALIZE_WHITESPACE</span><span class="p">,</span>
<span class="p">)</span>
<span class="n">shutil</span><span class="o">.</span><span class="n">rmtree</span><span class="p">(</span><span class="n">path</span><span class="p">,</span> <span class="n">ignore_errors</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="p">(</span><span class="s2">&quot;DROP DATABASE IF EXISTS </span><span class="si">%s</span><span class="s2"> CASCADE&quot;</span> <span class="o">%</span> <span class="n">db_name</span><span class="p">)</span>
<span class="n">spark</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>
</article>
<footer class="bd-footer-article">
<div class="footer-article-items footer-article__inner">
<div class="footer-article-item"><!-- Previous / next buttons -->
<div class="prev-next-area">
</div></div>
</div>
</footer>
</div>
</div>
<footer class="bd-footer-content">
</footer>
</main>
</div>
</div>
<!-- Scripts loaded after <body> so the DOM is not blocked -->
<script src="../../../_static/scripts/bootstrap.js?digest=e353d410970836974a52"></script>
<script src="../../../_static/scripts/pydata-sphinx-theme.js?digest=e353d410970836974a52"></script>
<footer class="bd-footer">
<div class="bd-footer__inner bd-page-width">
<div class="footer-items__start">
<div class="footer-item"><p class="copyright">
Copyright @ 2024 The Apache Software Foundation, Licensed under the <a href="https://www.apache.org/licenses/LICENSE-2.0">Apache License, Version 2.0</a>.
</p></div>
<div class="footer-item">
<p class="sphinx-version">
Created using <a href="https://www.sphinx-doc.org/">Sphinx</a> 4.5.0.
<br/>
</p>
</div>
</div>
<div class="footer-items__end">
<div class="footer-item"><p class="theme-version">
Built with the <a href="https://pydata-sphinx-theme.readthedocs.io/en/stable/index.html">PyData Sphinx Theme</a> 0.13.3.
</p></div>
</div>
</div>
</footer>
</body>
</html>