| |
| |
| <!DOCTYPE html> |
| |
| |
| <html > |
| |
| <head> |
| <meta charset="utf-8" /> |
| <meta name="viewport" content="width=device-width, initial-scale=1.0" /> |
| <title>pyspark.sql.udtf — 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/sql/udtf';</script> |
| <link rel="canonical" href="https://spark.apache.org/docs/latest/api/python/_modules/pyspark/sql/udtf.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/sql/udtf.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/sql/udtf.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.sql.udtf</li> |
| </ul> |
| </nav> |
| </div> |
| |
| </div> |
| |
| |
| </div> |
| </div> |
| |
| |
| |
| |
| <div id="searchbox"></div> |
| <article class="bd-article" role="main"> |
| |
| <h1>Source code for pyspark.sql.udtf</h1><div class="highlight"><pre> |
| <span></span><span class="c1">#</span> |
| <span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span> |
| <span class="c1"># contributor license agreements. See the NOTICE file distributed with</span> |
| <span class="c1"># this work for additional information regarding copyright ownership.</span> |
| <span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span> |
| <span class="c1"># (the "License"); you may not use this file except in compliance with</span> |
| <span class="c1"># the License. You may obtain a copy of the License at</span> |
| <span class="c1">#</span> |
| <span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span> |
| <span class="c1">#</span> |
| <span class="c1"># Unless required by applicable law or agreed to in writing, software</span> |
| <span class="c1"># distributed under the License is distributed on an "AS IS" BASIS,</span> |
| <span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span> |
| <span class="c1"># See the License for the specific language governing permissions and</span> |
| <span class="c1"># limitations under the License.</span> |
| <span class="c1">#</span> |
| <span class="sd">"""</span> |
| <span class="sd">User-defined table function related classes and functions</span> |
| <span class="sd">"""</span> |
| <span class="kn">import</span> <span class="nn">pickle</span> |
| <span class="kn">from</span> <span class="nn">dataclasses</span> <span class="kn">import</span> <span class="n">dataclass</span><span class="p">,</span> <span class="n">field</span> |
| <span class="kn">import</span> <span class="nn">inspect</span> |
| <span class="kn">import</span> <span class="nn">sys</span> |
| <span class="kn">import</span> <span class="nn">warnings</span> |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Any</span><span class="p">,</span> <span class="n">Type</span><span class="p">,</span> <span class="n">TYPE_CHECKING</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">Union</span> |
| |
| <span class="kn">from</span> <span class="nn">pyspark.errors</span> <span class="kn">import</span> <span class="n">PySparkAttributeError</span><span class="p">,</span> <span class="n">PySparkPicklingError</span><span class="p">,</span> <span class="n">PySparkTypeError</span> |
| <span class="kn">from</span> <span class="nn">pyspark.util</span> <span class="kn">import</span> <span class="n">PythonEvalType</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.pandas.utils</span> <span class="kn">import</span> <span class="n">require_minimum_pandas_version</span><span class="p">,</span> <span class="n">require_minimum_pyarrow_version</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.types</span> <span class="kn">import</span> <span class="n">DataType</span><span class="p">,</span> <span class="n">StructType</span><span class="p">,</span> <span class="n">_parse_datatype_string</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.udf</span> <span class="kn">import</span> <span class="n">_wrap_function</span> |
| |
| <span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span> |
| <span class="kn">from</span> <span class="nn">py4j.java_gateway</span> <span class="kn">import</span> <span class="n">JavaObject</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql._typing</span> <span class="kn">import</span> <span class="n">ColumnOrName</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.dataframe</span> <span class="kn">import</span> <span class="n">DataFrame</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.session</span> <span class="kn">import</span> <span class="n">SparkSession</span> |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span> |
| <span class="s2">"AnalyzeArgument"</span><span class="p">,</span> |
| <span class="s2">"AnalyzeResult"</span><span class="p">,</span> |
| <span class="s2">"PartitioningColumn"</span><span class="p">,</span> |
| <span class="s2">"OrderingColumn"</span><span class="p">,</span> |
| <span class="s2">"SelectedColumn"</span><span class="p">,</span> |
| <span class="s2">"SkipRestOfInputTableException"</span><span class="p">,</span> |
| <span class="s2">"UDTFRegistration"</span><span class="p">,</span> |
| <span class="p">]</span> |
| |
| |
| <span class="nd">@dataclass</span><span class="p">(</span><span class="n">frozen</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">AnalyzeArgument</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> The argument for Python UDTF's analyze static method.</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> dataType : :class:`DataType`</span> |
| <span class="sd"> The argument's data type</span> |
| <span class="sd"> value : any, optional</span> |
| <span class="sd"> The calculated value if the argument is foldable; otherwise None</span> |
| <span class="sd"> isTable : bool</span> |
| <span class="sd"> If True, the argument is a table argument.</span> |
| <span class="sd"> isConstantExpression : bool</span> |
| <span class="sd"> If True, the argument is a constant-foldable scalar expression. Then the 'value' field</span> |
| <span class="sd"> contains None if the argument is a NULL literal, or a non-None value if the argument is a</span> |
| <span class="sd"> non-NULL literal. In this way, we can distinguish between a literal NULL argument and other</span> |
| <span class="sd"> types of arguments such as complex expression trees or table arguments where the 'value'</span> |
| <span class="sd"> field is always None.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">dataType</span><span class="p">:</span> <span class="n">DataType</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="n">isTable</span><span class="p">:</span> <span class="nb">bool</span> |
| <span class="n">isConstantExpression</span><span class="p">:</span> <span class="nb">bool</span> |
| |
| |
| <span class="nd">@dataclass</span><span class="p">(</span><span class="n">frozen</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">PartitioningColumn</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Represents an expression that the UDTF is specifying for Catalyst to partition the input table</span> |
| <span class="sd"> by. This can be either the name of a single column from the input table (such as "columnA"), or</span> |
| <span class="sd"> a SQL expression based on the column names of the input table (such as "columnA + columnB").</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> name : str</span> |
| <span class="sd"> The contents of the partitioning column name or expression represented as a SQL string.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">name</span><span class="p">:</span> <span class="nb">str</span> |
| |
| |
| <span class="nd">@dataclass</span><span class="p">(</span><span class="n">frozen</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">OrderingColumn</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Represents an expression that the UDTF is specifying for Catalyst to order the input partition</span> |
| <span class="sd"> by. This can be either the name of a single column from the input table (such as "columnA"),</span> |
| <span class="sd"> or a SQL expression based on the column names of the input table (such as "columnA + columnB").</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> name : str</span> |
| <span class="sd"> The contents of the ordering column name or expression represented as a SQL string.</span> |
| <span class="sd"> ascending : bool, default True</span> |
| <span class="sd"> This is if this expression specifies an ascending sorting order.</span> |
| <span class="sd"> overrideNullsFirst : str, optional</span> |
| <span class="sd"> If this is None, use the default behavior to sort NULL values first when sorting in</span> |
| <span class="sd"> ascending order, or last when sorting in descending order. Otherwise, if this is</span> |
| <span class="sd"> True or False, we override the default behavior accordingly.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">name</span><span class="p">:</span> <span class="nb">str</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="n">overrideNullsFirst</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="nd">@dataclass</span><span class="p">(</span><span class="n">frozen</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span> |
| <span class="k">class</span> <span class="nc">SelectedColumn</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Represents an expression that the UDTF is specifying for Catalyst to evaluate against the</span> |
| <span class="sd"> columns in the input TABLE argument. The UDTF then receives one input column for each expression</span> |
| <span class="sd"> in the list, in the order they are listed.</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> name : str</span> |
| <span class="sd"> The contents of the selected column name or expression represented as a SQL string.</span> |
| <span class="sd"> alias : str, default ''</span> |
| <span class="sd"> If non-empty, this is the alias for the column or expression as visible from the UDTF's</span> |
| <span class="sd"> 'eval' method. This is required if the expression is not a simple column reference.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">name</span><span class="p">:</span> <span class="nb">str</span> |
| <span class="n">alias</span><span class="p">:</span> <span class="nb">str</span> <span class="o">=</span> <span class="s2">""</span> |
| |
| |
| <span class="c1"># Note: this class is a "dataclass" for purposes of convenience, but it is not marked "frozen"</span> |
| <span class="c1"># because the intention is that users may create subclasses of it for purposes of returning custom</span> |
| <span class="c1"># information from the "analyze" method.</span> |
| <span class="nd">@dataclass</span> |
| <span class="k">class</span> <span class="nc">AnalyzeResult</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> The return of Python UDTF's analyze static method.</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> schema: :class:`StructType`</span> |
| <span class="sd"> The schema that the Python UDTF will return.</span> |
| <span class="sd"> withSinglePartition: bool</span> |
| <span class="sd"> If true, the UDTF is specifying for Catalyst to repartition all rows of the input TABLE</span> |
| <span class="sd"> argument to one collection for consumption by exactly one instance of the correpsonding</span> |
| <span class="sd"> UDTF class.</span> |
| <span class="sd"> partitionBy: sequence of :class:`PartitioningColumn`</span> |
| <span class="sd"> If non-empty, this is a sequence of expressions that the UDTF is specifying for Catalyst to</span> |
| <span class="sd"> partition the input TABLE argument by. In this case, calls to the UDTF may not include any</span> |
| <span class="sd"> explicit PARTITION BY clause, in which case Catalyst will return an error. This option is</span> |
| <span class="sd"> mutually exclusive with 'withSinglePartition'.</span> |
| <span class="sd"> orderBy: sequence of :class:`OrderingColumn`</span> |
| <span class="sd"> If non-empty, this is a sequence of expressions that the UDTF is specifying for Catalyst to</span> |
| <span class="sd"> sort the input TABLE argument by. Note that the 'partitionBy' list must also be non-empty</span> |
| <span class="sd"> in this case.</span> |
| <span class="sd"> select: sequence of :class:`SelectedColumn`</span> |
| <span class="sd"> If non-empty, this is a sequence of expressions that the UDTF is specifying for Catalyst to</span> |
| <span class="sd"> evaluate against the columns in the input TABLE argument. The UDTF then receives one input</span> |
| <span class="sd"> attribute for each name in the list, in the order they are listed.</span> |
| <span class="sd"> """</span> |
| |
| <span class="n">schema</span><span class="p">:</span> <span class="n">StructType</span> |
| <span class="n">withSinglePartition</span><span class="p">:</span> <span class="nb">bool</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="n">partitionBy</span><span class="p">:</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">PartitioningColumn</span><span class="p">]</span> <span class="o">=</span> <span class="n">field</span><span class="p">(</span><span class="n">default_factory</span><span class="o">=</span><span class="nb">tuple</span><span class="p">)</span> |
| <span class="n">orderBy</span><span class="p">:</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">OrderingColumn</span><span class="p">]</span> <span class="o">=</span> <span class="n">field</span><span class="p">(</span><span class="n">default_factory</span><span class="o">=</span><span class="nb">tuple</span><span class="p">)</span> |
| <span class="n">select</span><span class="p">:</span> <span class="n">Sequence</span><span class="p">[</span><span class="n">SelectedColumn</span><span class="p">]</span> <span class="o">=</span> <span class="n">field</span><span class="p">(</span><span class="n">default_factory</span><span class="o">=</span><span class="nb">tuple</span><span class="p">)</span> |
| |
| |
| <span class="k">class</span> <span class="nc">SkipRestOfInputTableException</span><span class="p">(</span><span class="ne">Exception</span><span class="p">):</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> This represents an exception that the 'eval' method may raise to indicate that it is done</span> |
| <span class="sd"> consuming rows from the current partition of the input table. Then the UDTF's 'terminate'</span> |
| <span class="sd"> method runs (if any).</span> |
| <span class="sd"> """</span> |
| |
| <span class="k">pass</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_create_udtf</span><span class="p">(</span> |
| <span class="bp">cls</span><span class="p">:</span> <span class="n">Type</span><span class="p">,</span> |
| <span class="n">returnType</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">StructType</span><span class="p">,</span> <span class="nb">str</span><span class="p">]],</span> |
| <span class="n">name</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="n">evalType</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="n">PythonEvalType</span><span class="o">.</span><span class="n">SQL_TABLE_UDF</span><span class="p">,</span> |
| <span class="n">deterministic</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">-></span> <span class="s2">"UserDefinedTableFunction"</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""Create a Python UDTF with the given eval type."""</span> |
| <span class="n">udtf_obj</span> <span class="o">=</span> <span class="n">UserDefinedTableFunction</span><span class="p">(</span> |
| <span class="bp">cls</span><span class="p">,</span> <span class="n">returnType</span><span class="o">=</span><span class="n">returnType</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">evalType</span><span class="o">=</span><span class="n">evalType</span><span class="p">,</span> <span class="n">deterministic</span><span class="o">=</span><span class="n">deterministic</span> |
| <span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">udtf_obj</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_create_py_udtf</span><span class="p">(</span> |
| <span class="bp">cls</span><span class="p">:</span> <span class="n">Type</span><span class="p">,</span> |
| <span class="n">returnType</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">StructType</span><span class="p">,</span> <span class="nb">str</span><span class="p">]],</span> |
| <span class="n">name</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="n">deterministic</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">useArrow</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">-></span> <span class="s2">"UserDefinedTableFunction"</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""Create a regular or an Arrow-optimized Python UDTF."""</span> |
| <span class="c1"># Determine whether to create Arrow-optimized UDTFs.</span> |
| <span class="k">if</span> <span class="n">useArrow</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">arrow_enabled</span> <span class="o">=</span> <span class="n">useArrow</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">SparkSession</span> |
| |
| <span class="n">session</span> <span class="o">=</span> <span class="n">SparkSession</span><span class="o">.</span><span class="n">_instantiatedSession</span> |
| <span class="n">arrow_enabled</span> <span class="o">=</span> <span class="kc">False</span> |
| <span class="k">if</span> <span class="n">session</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">value</span> <span class="o">=</span> <span class="n">session</span><span class="o">.</span><span class="n">conf</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">"spark.sql.execution.pythonUDTF.arrow.enabled"</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">str</span><span class="p">)</span> <span class="ow">and</span> <span class="n">value</span><span class="o">.</span><span class="n">lower</span><span class="p">()</span> <span class="o">==</span> <span class="s2">"true"</span><span class="p">:</span> |
| <span class="n">arrow_enabled</span> <span class="o">=</span> <span class="kc">True</span> |
| |
| <span class="n">eval_type</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="n">PythonEvalType</span><span class="o">.</span><span class="n">SQL_TABLE_UDF</span> |
| |
| <span class="k">if</span> <span class="n">arrow_enabled</span><span class="p">:</span> |
| <span class="c1"># Return the regular UDTF if the required dependencies are not satisfied.</span> |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">require_minimum_pandas_version</span><span class="p">()</span> |
| <span class="n">require_minimum_pyarrow_version</span><span class="p">()</span> |
| <span class="n">eval_type</span> <span class="o">=</span> <span class="n">PythonEvalType</span><span class="o">.</span><span class="n">SQL_ARROW_TABLE_UDF</span> |
| <span class="k">except</span> <span class="ne">ImportError</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span> |
| <span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span> |
| <span class="sa">f</span><span class="s2">"Arrow optimization for Python UDTFs cannot be enabled: </span><span class="si">{</span><span class="nb">str</span><span class="p">(</span><span class="n">e</span><span class="p">)</span><span class="si">}</span><span class="s2">. "</span> |
| <span class="sa">f</span><span class="s2">"Falling back to using regular Python UDTFs."</span><span class="p">,</span> |
| <span class="ne">UserWarning</span><span class="p">,</span> |
| <span class="p">)</span> |
| |
| <span class="k">return</span> <span class="n">_create_udtf</span><span class="p">(</span> |
| <span class="bp">cls</span><span class="o">=</span><span class="bp">cls</span><span class="p">,</span> |
| <span class="n">returnType</span><span class="o">=</span><span class="n">returnType</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">evalType</span><span class="o">=</span><span class="n">eval_type</span><span class="p">,</span> |
| <span class="n">deterministic</span><span class="o">=</span><span class="n">deterministic</span><span class="p">,</span> |
| <span class="p">)</span> |
| |
| |
| <span class="k">def</span> <span class="nf">_validate_udtf_handler</span><span class="p">(</span><span class="bp">cls</span><span class="p">:</span> <span class="n">Any</span><span class="p">,</span> <span class="n">returnType</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">StructType</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="w"> </span><span class="sd">"""Validate the handler class of a UDTF."""</span> |
| |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="nb">type</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">PySparkTypeError</span><span class="p">(</span> |
| <span class="n">error_class</span><span class="o">=</span><span class="s2">"INVALID_UDTF_HANDLER_TYPE"</span><span class="p">,</span> <span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span><span class="s2">"type"</span><span class="p">:</span> <span class="nb">type</span><span class="p">(</span><span class="bp">cls</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="ow">not</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="s2">"eval"</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">PySparkAttributeError</span><span class="p">(</span> |
| <span class="n">error_class</span><span class="o">=</span><span class="s2">"INVALID_UDTF_NO_EVAL"</span><span class="p">,</span> <span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span><span class="s2">"name"</span><span class="p">:</span> <span class="bp">cls</span><span class="o">.</span><span class="vm">__name__</span><span class="p">}</span> |
| <span class="p">)</span> |
| |
| <span class="n">has_analyze</span> <span class="o">=</span> <span class="nb">hasattr</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="s2">"analyze"</span><span class="p">)</span> |
| <span class="n">has_analyze_staticmethod</span> <span class="o">=</span> <span class="n">has_analyze</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span> |
| <span class="n">inspect</span><span class="o">.</span><span class="n">getattr_static</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="s2">"analyze"</span><span class="p">),</span> <span class="nb">staticmethod</span> |
| <span class="p">)</span> |
| <span class="k">if</span> <span class="n">returnType</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">has_analyze_staticmethod</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">PySparkAttributeError</span><span class="p">(</span> |
| <span class="n">error_class</span><span class="o">=</span><span class="s2">"INVALID_UDTF_RETURN_TYPE"</span><span class="p">,</span> <span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span><span class="s2">"name"</span><span class="p">:</span> <span class="bp">cls</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="n">returnType</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> <span class="ow">and</span> <span class="n">has_analyze</span><span class="p">:</span> |
| <span class="k">raise</span> <span class="n">PySparkAttributeError</span><span class="p">(</span> |
| <span class="n">error_class</span><span class="o">=</span><span class="s2">"INVALID_UDTF_BOTH_RETURN_TYPE_AND_ANALYZE"</span><span class="p">,</span> |
| <span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span><span class="s2">"name"</span><span class="p">:</span> <span class="bp">cls</span><span class="o">.</span><span class="vm">__name__</span><span class="p">},</span> |
| <span class="p">)</span> |
| |
| |
| <div class="viewcode-block" id="UserDefinedTableFunction"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.udtf.UserDefinedTableFunction.html#pyspark.sql.UserDefinedTableFunction">[docs]</a><span class="k">class</span> <span class="nc">UserDefinedTableFunction</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> User-defined table function in Python</span> |
| |
| <span class="sd"> .. versionadded:: 3.5.0</span> |
| |
| <span class="sd"> Notes</span> |
| <span class="sd"> -----</span> |
| <span class="sd"> The constructor of this class is not supposed to be directly called.</span> |
| <span class="sd"> Use :meth:`pyspark.sql.functions.udtf` to create this instance.</span> |
| |
| <span class="sd"> This API is evolving.</span> |
| <span class="sd"> """</span> |
| |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span> |
| <span class="bp">self</span><span class="p">,</span> |
| <span class="n">func</span><span class="p">:</span> <span class="n">Type</span><span class="p">,</span> |
| <span class="n">returnType</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">StructType</span><span class="p">,</span> <span class="nb">str</span><span class="p">]],</span> |
| <span class="n">name</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="nb">str</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> |
| <span class="n">evalType</span><span class="p">:</span> <span class="nb">int</span> <span class="o">=</span> <span class="n">PythonEvalType</span><span class="o">.</span><span class="n">SQL_TABLE_UDF</span><span class="p">,</span> |
| <span class="n">deterministic</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="n">_validate_udtf_handler</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">returnType</span><span class="p">)</span> |
| |
| <span class="bp">self</span><span class="o">.</span><span class="n">func</span> <span class="o">=</span> <span class="n">func</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_returnType</span> <span class="o">=</span> <span class="n">returnType</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_returnType_placeholder</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">StructType</span><span class="p">]</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_inputTypes_placeholder</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_judtf_placeholder</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_name</span> <span class="o">=</span> <span class="n">name</span> <span class="ow">or</span> <span class="n">func</span><span class="o">.</span><span class="vm">__name__</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">evalType</span> <span class="o">=</span> <span class="n">evalType</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">deterministic</span> <span class="o">=</span> <span class="n">deterministic</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">returnType</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-></span> <span class="n">Optional</span><span class="p">[</span><span class="n">StructType</span><span class="p">]:</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_returnType</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="k">return</span> <span class="kc">None</span> |
| <span class="c1"># `_parse_datatype_string` accesses to JVM for parsing a DDL formatted string.</span> |
| <span class="c1"># This makes sure this is called after SparkContext is initialized.</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_returnType_placeholder</span> <span class="ow">is</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="bp">self</span><span class="o">.</span><span class="n">_returnType</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span> |
| <span class="n">parsed</span> <span class="o">=</span> <span class="n">_parse_datatype_string</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_returnType</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">parsed</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_returnType</span> |
| <span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">parsed</span><span class="p">,</span> <span class="n">StructType</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">PySparkTypeError</span><span class="p">(</span> |
| <span class="n">error_class</span><span class="o">=</span><span class="s2">"UDTF_RETURN_TYPE_MISMATCH"</span><span class="p">,</span> |
| <span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span> |
| <span class="s2">"name"</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_name</span><span class="p">,</span> |
| <span class="s2">"return_type"</span><span class="p">:</span> <span class="sa">f</span><span class="s2">"</span><span class="si">{</span><span class="n">parsed</span><span class="si">}</span><span class="s2">"</span><span class="p">,</span> |
| <span class="p">},</span> |
| <span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_returnType_placeholder</span> <span class="o">=</span> <span class="n">parsed</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_returnType_placeholder</span> |
| |
| <span class="nd">@property</span> |
| <span class="k">def</span> <span class="nf">_judtf</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-></span> <span class="s2">"JavaObject"</span><span class="p">:</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_judtf_placeholder</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_judtf_placeholder</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_create_judtf</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">func</span><span class="p">)</span> |
| <span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_judtf_placeholder</span> |
| |
| <span class="k">def</span> <span class="nf">_create_judtf</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">Type</span><span class="p">)</span> <span class="o">-></span> <span class="s2">"JavaObject"</span><span class="p">:</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">SparkSession</span> |
| |
| <span class="n">spark</span> <span class="o">=</span> <span class="n">SparkSession</span><span class="o">.</span><span class="n">_getActiveSessionOrCreate</span><span class="p">()</span> |
| <span class="n">sc</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span> |
| |
| <span class="k">try</span><span class="p">:</span> |
| <span class="n">wrapped_func</span> <span class="o">=</span> <span class="n">_wrap_function</span><span class="p">(</span><span class="n">sc</span><span class="p">,</span> <span class="n">func</span><span class="p">)</span> |
| <span class="k">except</span> <span class="n">pickle</span><span class="o">.</span><span class="n">PicklingError</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span> |
| <span class="k">if</span> <span class="s2">"CONTEXT_ONLY_VALID_ON_DRIVER"</span> <span class="ow">in</span> <span class="nb">str</span><span class="p">(</span><span class="n">e</span><span class="p">):</span> |
| <span class="k">raise</span> <span class="n">PySparkPicklingError</span><span class="p">(</span> |
| <span class="n">error_class</span><span class="o">=</span><span class="s2">"UDTF_SERIALIZATION_ERROR"</span><span class="p">,</span> |
| <span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span> |
| <span class="s2">"name"</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_name</span><span class="p">,</span> |
| <span class="s2">"message"</span><span class="p">:</span> <span class="s2">"it appears that you are attempting to reference SparkSession "</span> |
| <span class="s2">"inside a UDTF. SparkSession can only be used on the driver, "</span> |
| <span class="s2">"not in code that runs on workers. Please remove the reference "</span> |
| <span class="s2">"and try again."</span><span class="p">,</span> |
| <span class="p">},</span> |
| <span class="p">)</span> <span class="kn">from</span> <span class="kc">None</span> |
| <span class="k">raise</span> <span class="n">PySparkPicklingError</span><span class="p">(</span> |
| <span class="n">error_class</span><span class="o">=</span><span class="s2">"UDTF_SERIALIZATION_ERROR"</span><span class="p">,</span> |
| <span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span> |
| <span class="s2">"name"</span><span class="p">:</span> <span class="bp">self</span><span class="o">.</span><span class="n">_name</span><span class="p">,</span> |
| <span class="s2">"message"</span><span class="p">:</span> <span class="s2">"Please check the stack trace and make sure the "</span> |
| <span class="s2">"function is serializable."</span><span class="p">,</span> |
| <span class="p">},</span> |
| <span class="p">)</span> |
| |
| <span class="k">assert</span> <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> |
| <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">returnType</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">judtf</span> <span class="o">=</span> <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">org</span><span class="o">.</span><span class="n">apache</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">execution</span><span class="o">.</span><span class="n">python</span><span class="o">.</span><span class="n">UserDefinedPythonTableFunction</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_name</span><span class="p">,</span> <span class="n">wrapped_func</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">evalType</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">deterministic</span> |
| <span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">jdt</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">_jsparkSession</span><span class="o">.</span><span class="n">parseDataType</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">returnType</span><span class="o">.</span><span class="n">json</span><span class="p">())</span> |
| <span class="n">judtf</span> <span class="o">=</span> <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">org</span><span class="o">.</span><span class="n">apache</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">execution</span><span class="o">.</span><span class="n">python</span><span class="o">.</span><span class="n">UserDefinedPythonTableFunction</span><span class="p">(</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_name</span><span class="p">,</span> <span class="n">wrapped_func</span><span class="p">,</span> <span class="n">jdt</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">evalType</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">deterministic</span> |
| <span class="p">)</span> |
| <span class="k">return</span> <span class="n">judtf</span> |
| |
| <span class="k">def</span> <span class="fm">__call__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">args</span><span class="p">:</span> <span class="s2">"ColumnOrName"</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">:</span> <span class="s2">"ColumnOrName"</span><span class="p">)</span> <span class="o">-></span> <span class="s2">"DataFrame"</span><span class="p">:</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.classic.column</span> <span class="kn">import</span> <span class="n">_to_java_column</span><span class="p">,</span> <span class="n">_to_java_expr</span><span class="p">,</span> <span class="n">_to_seq</span> |
| |
| <span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">DataFrame</span><span class="p">,</span> <span class="n">SparkSession</span> |
| |
| <span class="n">spark</span> <span class="o">=</span> <span class="n">SparkSession</span><span class="o">.</span><span class="n">_getActiveSessionOrCreate</span><span class="p">()</span> |
| <span class="n">sc</span> <span class="o">=</span> <span class="n">spark</span><span class="o">.</span><span class="n">sparkContext</span> |
| |
| <span class="k">assert</span> <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span> |
| <span class="n">jcols</span> <span class="o">=</span> <span class="p">[</span><span class="n">_to_java_column</span><span class="p">(</span><span class="n">arg</span><span class="p">)</span> <span class="k">for</span> <span class="n">arg</span> <span class="ow">in</span> <span class="n">args</span><span class="p">]</span> <span class="o">+</span> <span class="p">[</span> |
| <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">Column</span><span class="p">(</span> |
| <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">org</span><span class="o">.</span><span class="n">apache</span><span class="o">.</span><span class="n">spark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">catalyst</span><span class="o">.</span><span class="n">expressions</span><span class="o">.</span><span class="n">NamedArgumentExpression</span><span class="p">(</span> |
| <span class="n">key</span><span class="p">,</span> <span class="n">_to_java_expr</span><span class="p">(</span><span class="n">value</span><span class="p">)</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">kwargs</span><span class="o">.</span><span class="n">items</span><span class="p">()</span> |
| <span class="p">]</span> |
| |
| <span class="n">judtf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_judtf</span> |
| <span class="n">jPythonUDTF</span> <span class="o">=</span> <span class="n">judtf</span><span class="o">.</span><span class="n">apply</span><span class="p">(</span><span class="n">spark</span><span class="o">.</span><span class="n">_jsparkSession</span><span class="p">,</span> <span class="n">_to_seq</span><span class="p">(</span><span class="n">sc</span><span class="p">,</span> <span class="n">jcols</span><span class="p">))</span> |
| <span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">jPythonUDTF</span><span class="p">,</span> <span class="n">spark</span><span class="p">)</span> |
| |
| <div class="viewcode-block" id="UserDefinedTableFunction.asDeterministic"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.udtf.UserDefinedTableFunction.asDeterministic.html#pyspark.sql.UserDefinedTableFunction.asDeterministic">[docs]</a> <span class="k">def</span> <span class="nf">asDeterministic</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-></span> <span class="s2">"UserDefinedTableFunction"</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Updates UserDefinedTableFunction to deterministic.</span> |
| <span class="sd"> """</span> |
| <span class="c1"># Explicitly clean the cache to create a JVM UDTF instance.</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_judtf_placeholder</span> <span class="o">=</span> <span class="kc">None</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">deterministic</span> <span class="o">=</span> <span class="kc">True</span> |
| <span class="k">return</span> <span class="bp">self</span></div></div> |
| |
| |
| <div class="viewcode-block" id="UDTFRegistration"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.UDTFRegistration.html#pyspark.sql.UDTFRegistration">[docs]</a><span class="k">class</span> <span class="nc">UDTFRegistration</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""</span> |
| <span class="sd"> Wrapper for user-defined table function registration. This instance can be accessed by</span> |
| <span class="sd"> :attr:`spark.udtf` or :attr:`sqlContext.udtf`.</span> |
| |
| <span class="sd"> .. versionadded:: 3.5.0</span> |
| <span class="sd"> """</span> |
| |
| <span class="k">def</span> <span class="fm">__init__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">sparkSession</span><span class="p">:</span> <span class="s2">"SparkSession"</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">sparkSession</span> <span class="o">=</span> <span class="n">sparkSession</span> |
| |
| <div class="viewcode-block" id="UDTFRegistration.register"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.UDTFRegistration.register.html#pyspark.sql.UDTFRegistration.register">[docs]</a> <span class="k">def</span> <span class="nf">register</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="n">f</span><span class="p">:</span> <span class="s2">"UserDefinedTableFunction"</span><span class="p">,</span> |
| <span class="p">)</span> <span class="o">-></span> <span class="s2">"UserDefinedTableFunction"</span><span class="p">:</span> |
| <span class="w"> </span><span class="sd">"""Register a Python user-defined table function as a SQL table function.</span> |
| |
| <span class="sd"> .. versionadded:: 3.5.0</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> name : str</span> |
| <span class="sd"> The name of the user-defined table function in SQL statements.</span> |
| <span class="sd"> f : function or :meth:`pyspark.sql.functions.udtf`</span> |
| <span class="sd"> The user-defined table function.</span> |
| |
| <span class="sd"> Returns</span> |
| <span class="sd"> -------</span> |
| <span class="sd"> function</span> |
| <span class="sd"> The registered user-defined table function.</span> |
| |
| <span class="sd"> Notes</span> |
| <span class="sd"> -----</span> |
| <span class="sd"> Spark uses the return type of the given user-defined table function as the return</span> |
| <span class="sd"> type of the registered user-defined function.</span> |
| |
| <span class="sd"> To register a nondeterministic Python table function, users need to first build</span> |
| <span class="sd"> a nondeterministic user-defined table function and then register it as a SQL function.</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> >>> from pyspark.sql.functions import udtf</span> |
| <span class="sd"> >>> @udtf(returnType="c1: int, c2: int")</span> |
| <span class="sd"> ... class PlusOne:</span> |
| <span class="sd"> ... def eval(self, x: int):</span> |
| <span class="sd"> ... yield x, x + 1</span> |
| <span class="sd"> ...</span> |
| <span class="sd"> >>> _ = spark.udtf.register(name="plus_one", f=PlusOne)</span> |
| <span class="sd"> >>> spark.sql("SELECT * FROM plus_one(1)").collect()</span> |
| <span class="sd"> [Row(c1=1, c2=2)]</span> |
| |
| <span class="sd"> Use it with lateral join</span> |
| |
| <span class="sd"> >>> spark.sql("SELECT * FROM VALUES (0, 1), (1, 2) t(x, y), LATERAL plus_one(x)").collect()</span> |
| <span class="sd"> [Row(x=0, y=1, c1=0, c2=1), Row(x=1, y=2, c1=1, c2=2)]</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="n">f</span><span class="o">.</span><span class="n">evalType</span> <span class="ow">not</span> <span class="ow">in</span> <span class="p">[</span><span class="n">PythonEvalType</span><span class="o">.</span><span class="n">SQL_TABLE_UDF</span><span class="p">,</span> <span class="n">PythonEvalType</span><span class="o">.</span><span class="n">SQL_ARROW_TABLE_UDF</span><span class="p">]:</span> |
| <span class="k">raise</span> <span class="n">PySparkTypeError</span><span class="p">(</span> |
| <span class="n">error_class</span><span class="o">=</span><span class="s2">"INVALID_UDTF_EVAL_TYPE"</span><span class="p">,</span> |
| <span class="n">message_parameters</span><span class="o">=</span><span class="p">{</span> |
| <span class="s2">"name"</span><span class="p">:</span> <span class="n">name</span><span class="p">,</span> |
| <span class="s2">"eval_type"</span><span class="p">:</span> <span class="s2">"SQL_TABLE_UDF, SQL_ARROW_TABLE_UDF"</span><span class="p">,</span> |
| <span class="p">},</span> |
| <span class="p">)</span> |
| |
| <span class="n">register_udtf</span> <span class="o">=</span> <span class="n">_create_udtf</span><span class="p">(</span> |
| <span class="bp">cls</span><span class="o">=</span><span class="n">f</span><span class="o">.</span><span class="n">func</span><span class="p">,</span> |
| <span class="n">returnType</span><span class="o">=</span><span class="n">f</span><span class="o">.</span><span class="n">returnType</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">evalType</span><span class="o">=</span><span class="n">f</span><span class="o">.</span><span class="n">evalType</span><span class="p">,</span> |
| <span class="n">deterministic</span><span class="o">=</span><span class="n">f</span><span class="o">.</span><span class="n">deterministic</span><span class="p">,</span> |
| <span class="p">)</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">sparkSession</span><span class="o">.</span><span class="n">_jsparkSession</span><span class="o">.</span><span class="n">udtf</span><span class="p">()</span><span class="o">.</span><span class="n">registerPython</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">register_udtf</span><span class="o">.</span><span class="n">_judtf</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">register_udtf</span></div></div> |
| |
| |
| <span class="k">def</span> <span class="nf">_test</span><span class="p">()</span> <span class="o">-></span> <span class="kc">None</span><span class="p">:</span> |
| <span class="kn">import</span> <span class="nn">doctest</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql</span> <span class="kn">import</span> <span class="n">SparkSession</span> |
| <span class="kn">import</span> <span class="nn">pyspark.sql.udf</span> |
| |
| <span class="n">globs</span> <span class="o">=</span> <span class="n">pyspark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">udtf</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">spark</span> <span class="o">=</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">"local[4]"</span><span class="p">)</span><span class="o">.</span><span class="n">appName</span><span class="p">(</span><span class="s2">"sql.udtf tests"</span><span class="p">)</span><span class="o">.</span><span class="n">getOrCreate</span><span class="p">()</span> |
| <span class="n">globs</span><span class="p">[</span><span class="s2">"spark"</span><span class="p">]</span> <span class="o">=</span> <span class="n">spark</span> |
| <span class="p">(</span><span class="n">failure_count</span><span class="p">,</span> <span class="n">test_count</span><span class="p">)</span> <span class="o">=</span> <span class="n">doctest</span><span class="o">.</span><span class="n">testmod</span><span class="p">(</span> |
| <span class="n">pyspark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">udtf</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="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">"__main__"</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> |