blob: 286a061a1b1539cd296aa4b12aa2cbb981bd9c24 [file] [log] [blame]
<!DOCTYPE html>
<html>
<head>
<meta charset="utf-8" />
<title>pyspark.sql.column &#8212; PySpark 3.1.1 documentation</title>
<link rel="stylesheet" href="../../../_static/css/index.73d71520a4ca3b99cfee5594769eaaae.css">
<link rel="stylesheet"
href="../../../_static/vendor/fontawesome/5.13.0/css/all.min.css">
<link rel="preload" as="font" type="font/woff2" crossorigin
href="../../../_static/vendor/fontawesome/5.13.0/webfonts/fa-solid-900.woff2">
<link rel="preload" as="font" type="font/woff2" crossorigin
href="../../../_static/vendor/fontawesome/5.13.0/webfonts/fa-brands-400.woff2">
<link rel="stylesheet"
href="../../../_static/vendor/open-sans_all/1.44.1/index.css">
<link rel="stylesheet"
href="../../../_static/vendor/lato_latin-ext/1.44.1/index.css">
<link rel="stylesheet" href="../../../_static/basic.css" type="text/css" />
<link rel="stylesheet" href="../../../_static/pygments.css" type="text/css" />
<link rel="stylesheet" type="text/css" href="../../../_static/css/pyspark.css" />
<link rel="preload" as="script" href="../../../_static/js/index.3da636dd464baa7582d2.js">
<script id="documentation_options" data-url_root="../../../" src="../../../_static/documentation_options.js"></script>
<script src="../../../_static/jquery.js"></script>
<script src="../../../_static/underscore.js"></script>
<script src="../../../_static/doctools.js"></script>
<script src="../../../_static/language_data.js"></script>
<script src="../../../_static/copybutton.js"></script>
<script crossorigin="anonymous" integrity="sha256-Ae2Vz/4ePdIu6ZyI/5ZGsYnb+m0JlOmKPjt6XZ9JJkA=" src="https://cdnjs.cloudflare.com/ajax/libs/require.js/2.3.4/require.min.js"></script>
<script async="async" src="https://cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.5/latest.js?config=TeX-AMS-MML_HTMLorMML"></script>
<script type="text/x-mathjax-config">MathJax.Hub.Config({"tex2jax": {"inlineMath": [["$", "$"], ["\\(", "\\)"]], "processEscapes": true, "ignoreClass": "document", "processClass": "math|output_area"}})</script>
<link rel="canonical" href="https://spark.apache.org/docs/latest/api/python/_modules/pyspark/sql/column.html" />
<link rel="search" title="Search" href="../../../search.html" />
<meta name="viewport" content="width=device-width, initial-scale=1" />
<meta name="docsearch:language" content="en" />
<!-- 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-spy="scroll" data-target="#bd-toc-nav" data-offset="80">
<nav class="navbar navbar-light navbar-expand-lg bg-light fixed-top bd-navbar" id="navbar-main">
<div class="container-xl">
<a class="navbar-brand" href="../../../index.html">
<img src="../../../_static/spark-logo-reverse.png" class="logo" alt="logo" />
</a>
<button class="navbar-toggler" type="button" data-toggle="collapse" data-target="#navbar-menu" aria-controls="navbar-menu" aria-expanded="false" aria-label="Toggle navigation">
<span class="navbar-toggler-icon"></span>
</button>
<div id="navbar-menu" class="col-lg-9 collapse navbar-collapse">
<ul id="navbar-main-elements" class="navbar-nav mr-auto">
<li class="nav-item ">
<a class="nav-link" href="../../../getting_started/index.html">Getting Started</a>
</li>
<li class="nav-item ">
<a class="nav-link" href="../../../user_guide/index.html">User Guide</a>
</li>
<li class="nav-item ">
<a class="nav-link" href="../../../reference/index.html">API Reference</a>
</li>
<li class="nav-item ">
<a class="nav-link" href="../../../development/index.html">Development</a>
</li>
<li class="nav-item ">
<a class="nav-link" href="../../../migration_guide/index.html">Migration Guide</a>
</li>
</ul>
<ul class="navbar-nav">
</ul>
</div>
</div>
</nav>
<div class="container-xl">
<div class="row">
<div class="col-12 col-md-3 bd-sidebar"><form class="bd-search d-flex align-items-center" action="../../../search.html" method="get">
<i class="icon fas fa-search"></i>
<input type="search" class="form-control" name="q" id="search-input" placeholder="Search the docs ..." aria-label="Search the docs ..." autocomplete="off" >
</form>
<nav class="bd-links" id="bd-docs-nav" aria-label="Main navigation">
<div class="bd-toc-item active">
<ul class="nav bd-sidenav">
</ul>
</nav>
</div>
<div class="d-none d-xl-block col-xl-2 bd-toc">
<nav id="bd-toc-nav">
<ul class="nav section-nav flex-column">
</ul>
</nav>
</div>
<main class="col-12 col-md-9 col-xl-7 py-md-5 pl-md-5 pr-md-4 bd-content" role="main">
<div>
<h1>Source code for pyspark.sql.column</h1><div class="highlight"><pre>
<span></span><span class="c1">#</span>
<span class="c1"># Licensed to the Apache Software Foundation (ASF) under one or more</span>
<span class="c1"># contributor license agreements. See the NOTICE file distributed with</span>
<span class="c1"># this work for additional information regarding copyright ownership.</span>
<span class="c1"># The ASF licenses this file to You under the Apache License, Version 2.0</span>
<span class="c1"># (the &quot;License&quot;); you may not use this file except in compliance with</span>
<span class="c1"># the License. You may obtain a copy of the License at</span>
<span class="c1">#</span>
<span class="c1"># http://www.apache.org/licenses/LICENSE-2.0</span>
<span class="c1">#</span>
<span class="c1"># Unless required by applicable law or agreed to in writing, software</span>
<span class="c1"># distributed under the License is distributed on an &quot;AS IS&quot; BASIS,</span>
<span class="c1"># WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.</span>
<span class="c1"># See the License for the specific language governing permissions and</span>
<span class="c1"># limitations under the License.</span>
<span class="c1">#</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">import</span> <span class="nn">json</span>
<span class="kn">import</span> <span class="nn">warnings</span>
<span class="kn">from</span> <span class="nn">pyspark</span> <span class="kn">import</span> <span class="n">copy_func</span>
<span class="kn">from</span> <span class="nn">pyspark.context</span> <span class="kn">import</span> <span class="n">SparkContext</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.types</span> <span class="kn">import</span> <span class="n">DataType</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">IntegerType</span><span class="p">,</span> <span class="n">StringType</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;Column&quot;</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">_create_column_from_literal</span><span class="p">(</span><span class="n">literal</span><span class="p">):</span>
<span class="n">sc</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="o">.</span><span class="n">_active_spark_context</span>
<span class="k">return</span> <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">functions</span><span class="o">.</span><span class="n">lit</span><span class="p">(</span><span class="n">literal</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_create_column_from_name</span><span class="p">(</span><span class="n">name</span><span class="p">):</span>
<span class="n">sc</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="o">.</span><span class="n">_active_spark_context</span>
<span class="k">return</span> <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">functions</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">def</span> <span class="nf">_to_java_column</span><span class="p">(</span><span class="n">col</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">col</span><span class="p">,</span> <span class="n">Column</span><span class="p">):</span>
<span class="n">jcol</span> <span class="o">=</span> <span class="n">col</span><span class="o">.</span><span class="n">_jc</span>
<span class="k">elif</span> <span class="nb">isinstance</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="n">jcol</span> <span class="o">=</span> <span class="n">_create_column_from_name</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">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;Invalid argument, not a string or column: &quot;</span>
<span class="s2">&quot;</span><span class="si">{0}</span><span class="s2"> of type </span><span class="si">{1}</span><span class="s2">. &quot;</span>
<span class="s2">&quot;For column literals, use &#39;lit&#39;, &#39;array&#39;, &#39;struct&#39; or &#39;create_map&#39; &quot;</span>
<span class="s2">&quot;function.&quot;</span><span class="o">.</span><span class="n">format</span><span class="p">(</span><span class="n">col</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="k">return</span> <span class="n">jcol</span>
<span class="k">def</span> <span class="nf">_to_seq</span><span class="p">(</span><span class="n">sc</span><span class="p">,</span> <span class="n">cols</span><span class="p">,</span> <span class="n">converter</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Convert a list of Column (or names) into a JVM Seq of Column.</span>
<span class="sd"> An optional `converter` could be used to convert items in `cols`</span>
<span class="sd"> into JVM Column objects.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">converter</span><span class="p">:</span>
<span class="n">cols</span> <span class="o">=</span> <span class="p">[</span><span class="n">converter</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">cols</span><span class="p">]</span>
<span class="k">return</span> <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonUtils</span><span class="o">.</span><span class="n">toSeq</span><span class="p">(</span><span class="n">cols</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_to_list</span><span class="p">(</span><span class="n">sc</span><span class="p">,</span> <span class="n">cols</span><span class="p">,</span> <span class="n">converter</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Convert a list of Column (or names) into a JVM (Scala) List of Column.</span>
<span class="sd"> An optional `converter` could be used to convert items in `cols`</span>
<span class="sd"> into JVM Column objects.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">converter</span><span class="p">:</span>
<span class="n">cols</span> <span class="o">=</span> <span class="p">[</span><span class="n">converter</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">cols</span><span class="p">]</span>
<span class="k">return</span> <span class="n">sc</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonUtils</span><span class="o">.</span><span class="n">toList</span><span class="p">(</span><span class="n">cols</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_unary_op</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">doc</span><span class="o">=</span><span class="s2">&quot;unary operator&quot;</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot; Create a method for given unary operator &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">_</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">jc</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="p">,</span> <span class="n">name</span><span class="p">)()</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span>
<span class="n">_</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">doc</span>
<span class="k">return</span> <span class="n">_</span>
<span class="k">def</span> <span class="nf">_func_op</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">doc</span><span class="o">=</span><span class="s1">&#39;&#39;</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">_</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="n">sc</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="o">.</span><span class="n">_active_spark_context</span>
<span class="n">jc</span> <span class="o">=</span> <span class="nb">getattr</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">functions</span><span class="p">,</span> <span class="n">name</span><span class="p">)(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span>
<span class="n">_</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">doc</span>
<span class="k">return</span> <span class="n">_</span>
<span class="k">def</span> <span class="nf">_bin_func_op</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">doc</span><span class="o">=</span><span class="s2">&quot;binary function&quot;</span><span class="p">):</span>
<span class="k">def</span> <span class="nf">_</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">sc</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="o">.</span><span class="n">_active_spark_context</span>
<span class="n">fn</span> <span class="o">=</span> <span class="nb">getattr</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">functions</span><span class="p">,</span> <span class="n">name</span><span class="p">)</span>
<span class="n">jc</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">_jc</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">Column</span><span class="p">)</span> <span class="k">else</span> <span class="n">_create_column_from_literal</span><span class="p">(</span><span class="n">other</span><span class="p">)</span>
<span class="n">njc</span> <span class="o">=</span> <span class="n">fn</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="p">,</span> <span class="n">jc</span><span class="p">)</span> <span class="k">if</span> <span class="ow">not</span> <span class="n">reverse</span> <span class="k">else</span> <span class="n">fn</span><span class="p">(</span><span class="n">jc</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">njc</span><span class="p">)</span>
<span class="n">_</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">doc</span>
<span class="k">return</span> <span class="n">_</span>
<span class="k">def</span> <span class="nf">_bin_op</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">doc</span><span class="o">=</span><span class="s2">&quot;binary operator&quot;</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot; Create a method for given binary operator</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">_</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">jc</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">_jc</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">Column</span><span class="p">)</span> <span class="k">else</span> <span class="n">other</span>
<span class="n">njc</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="p">,</span> <span class="n">name</span><span class="p">)(</span><span class="n">jc</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">njc</span><span class="p">)</span>
<span class="n">_</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">doc</span>
<span class="k">return</span> <span class="n">_</span>
<span class="k">def</span> <span class="nf">_reverse_op</span><span class="p">(</span><span class="n">name</span><span class="p">,</span> <span class="n">doc</span><span class="o">=</span><span class="s2">&quot;binary operator&quot;</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot; Create a method for binary operator (this object is on right side)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">_</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">jother</span> <span class="o">=</span> <span class="n">_create_column_from_literal</span><span class="p">(</span><span class="n">other</span><span class="p">)</span>
<span class="n">jc</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="n">jother</span><span class="p">,</span> <span class="n">name</span><span class="p">)(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span>
<span class="n">_</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">doc</span>
<span class="k">return</span> <span class="n">_</span>
<div class="viewcode-block" id="Column"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.html#pyspark.sql.Column">[docs]</a><span class="k">class</span> <span class="nc">Column</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> A column in a DataFrame.</span>
<span class="sd"> :class:`Column` instances can be created by::</span>
<span class="sd"> # 1. Select a column out of a DataFrame</span>
<span class="sd"> df.colName</span>
<span class="sd"> df[&quot;colName&quot;]</span>
<span class="sd"> # 2. Create from an expression</span>
<span class="sd"> df.colName + 1</span>
<span class="sd"> 1 / df.colName</span>
<span class="sd"> .. versionadded:: 1.3.0</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">jc</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jc</span> <span class="o">=</span> <span class="n">jc</span>
<span class="c1"># arithmetic operators</span>
<span class="fm">__neg__</span> <span class="o">=</span> <span class="n">_func_op</span><span class="p">(</span><span class="s2">&quot;negate&quot;</span><span class="p">)</span>
<span class="fm">__add__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;plus&quot;</span><span class="p">)</span>
<span class="fm">__sub__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;minus&quot;</span><span class="p">)</span>
<span class="fm">__mul__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;multiply&quot;</span><span class="p">)</span>
<span class="n">__div__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;divide&quot;</span><span class="p">)</span>
<span class="fm">__truediv__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;divide&quot;</span><span class="p">)</span>
<span class="fm">__mod__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;mod&quot;</span><span class="p">)</span>
<span class="fm">__radd__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;plus&quot;</span><span class="p">)</span>
<span class="fm">__rsub__</span> <span class="o">=</span> <span class="n">_reverse_op</span><span class="p">(</span><span class="s2">&quot;minus&quot;</span><span class="p">)</span>
<span class="fm">__rmul__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;multiply&quot;</span><span class="p">)</span>
<span class="n">__rdiv__</span> <span class="o">=</span> <span class="n">_reverse_op</span><span class="p">(</span><span class="s2">&quot;divide&quot;</span><span class="p">)</span>
<span class="fm">__rtruediv__</span> <span class="o">=</span> <span class="n">_reverse_op</span><span class="p">(</span><span class="s2">&quot;divide&quot;</span><span class="p">)</span>
<span class="fm">__rmod__</span> <span class="o">=</span> <span class="n">_reverse_op</span><span class="p">(</span><span class="s2">&quot;mod&quot;</span><span class="p">)</span>
<span class="fm">__pow__</span> <span class="o">=</span> <span class="n">_bin_func_op</span><span class="p">(</span><span class="s2">&quot;pow&quot;</span><span class="p">)</span>
<span class="fm">__rpow__</span> <span class="o">=</span> <span class="n">_bin_func_op</span><span class="p">(</span><span class="s2">&quot;pow&quot;</span><span class="p">,</span> <span class="n">reverse</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="c1"># logistic operators</span>
<span class="fm">__eq__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;equalTo&quot;</span><span class="p">)</span>
<span class="fm">__ne__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;notEqual&quot;</span><span class="p">)</span>
<span class="fm">__lt__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;lt&quot;</span><span class="p">)</span>
<span class="fm">__le__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;leq&quot;</span><span class="p">)</span>
<span class="fm">__ge__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;geq&quot;</span><span class="p">)</span>
<span class="fm">__gt__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;gt&quot;</span><span class="p">)</span>
<span class="n">_eqNullSafe_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Equality test that is safe for null values.</span>
<span class="s2"> .. versionadded:: 2.3.0</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other</span>
<span class="s2"> a value or :class:`Column`</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df1 = spark.createDataFrame([</span>
<span class="s2"> ... Row(id=1, value=&#39;foo&#39;),</span>
<span class="s2"> ... Row(id=2, value=None)</span>
<span class="s2"> ... ])</span>
<span class="s2"> &gt;&gt;&gt; df1.select(</span>
<span class="s2"> ... df1[&#39;value&#39;] == &#39;foo&#39;,</span>
<span class="s2"> ... df1[&#39;value&#39;].eqNullSafe(&#39;foo&#39;),</span>
<span class="s2"> ... df1[&#39;value&#39;].eqNullSafe(None)</span>
<span class="s2"> ... ).show()</span>
<span class="s2"> +-------------+---------------+----------------+</span>
<span class="s2"> |(value = foo)|(value &lt;=&gt; foo)|(value &lt;=&gt; NULL)|</span>
<span class="s2"> +-------------+---------------+----------------+</span>
<span class="s2"> | true| true| false|</span>
<span class="s2"> | null| false| true|</span>
<span class="s2"> +-------------+---------------+----------------+</span>
<span class="s2"> &gt;&gt;&gt; df2 = spark.createDataFrame([</span>
<span class="s2"> ... Row(value = &#39;bar&#39;),</span>
<span class="s2"> ... Row(value = None)</span>
<span class="s2"> ... ])</span>
<span class="s2"> &gt;&gt;&gt; df1.join(df2, df1[&quot;value&quot;] == df2[&quot;value&quot;]).count()</span>
<span class="s2"> 0</span>
<span class="s2"> &gt;&gt;&gt; df1.join(df2, df1[&quot;value&quot;].eqNullSafe(df2[&quot;value&quot;])).count()</span>
<span class="s2"> 1</span>
<span class="s2"> &gt;&gt;&gt; df2 = spark.createDataFrame([</span>
<span class="s2"> ... Row(id=1, value=float(&#39;NaN&#39;)),</span>
<span class="s2"> ... Row(id=2, value=42.0),</span>
<span class="s2"> ... Row(id=3, value=None)</span>
<span class="s2"> ... ])</span>
<span class="s2"> &gt;&gt;&gt; df2.select(</span>
<span class="s2"> ... df2[&#39;value&#39;].eqNullSafe(None),</span>
<span class="s2"> ... df2[&#39;value&#39;].eqNullSafe(float(&#39;NaN&#39;)),</span>
<span class="s2"> ... df2[&#39;value&#39;].eqNullSafe(42.0)</span>
<span class="s2"> ... ).show()</span>
<span class="s2"> +----------------+---------------+----------------+</span>
<span class="s2"> |(value &lt;=&gt; NULL)|(value &lt;=&gt; NaN)|(value &lt;=&gt; 42.0)|</span>
<span class="s2"> +----------------+---------------+----------------+</span>
<span class="s2"> | false| true| false|</span>
<span class="s2"> | false| false| true|</span>
<span class="s2"> | true| false| false|</span>
<span class="s2"> +----------------+---------------+----------------+</span>
<span class="s2"> Notes</span>
<span class="s2"> -----</span>
<span class="s2"> Unlike Pandas, PySpark doesn&#39;t consider NaN values to be NULL. See the</span>
<span class="s2"> `NaN Semantics &lt;https://spark.apache.org/docs/latest/sql-ref-datatypes.html#nan-semantics&gt;`_</span>
<span class="s2"> for details.</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">eqNullSafe</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;eqNullSafe&quot;</span><span class="p">,</span> <span class="n">_eqNullSafe_doc</span><span class="p">)</span>
<span class="c1"># `and`, `or`, `not` cannot be overloaded in Python,</span>
<span class="c1"># so use bitwise operators as boolean operators</span>
<span class="fm">__and__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s1">&#39;and&#39;</span><span class="p">)</span>
<span class="fm">__or__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s1">&#39;or&#39;</span><span class="p">)</span>
<span class="fm">__invert__</span> <span class="o">=</span> <span class="n">_func_op</span><span class="p">(</span><span class="s1">&#39;not&#39;</span><span class="p">)</span>
<span class="fm">__rand__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;and&quot;</span><span class="p">)</span>
<span class="fm">__ror__</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;or&quot;</span><span class="p">)</span>
<span class="c1"># container operators</span>
<span class="k">def</span> <span class="fm">__contains__</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="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Cannot apply &#39;in&#39; operator against a column: please use &#39;contains&#39; &quot;</span>
<span class="s2">&quot;in a string column or &#39;array_contains&#39; function for an array column.&quot;</span><span class="p">)</span>
<span class="c1"># bitwise operators</span>
<span class="n">_bitwiseOR_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Compute bitwise OR of this expression with another expression.</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other</span>
<span class="s2"> a value or :class:`Column` to calculate bitwise or(|) with</span>
<span class="s2"> this :class:`Column`.</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([Row(a=170, b=75)])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.a.bitwiseOR(df.b)).collect()</span>
<span class="s2"> [Row((a | b)=235)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_bitwiseAND_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Compute bitwise AND of this expression with another expression.</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other</span>
<span class="s2"> a value or :class:`Column` to calculate bitwise and(&amp;) with</span>
<span class="s2"> this :class:`Column`.</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([Row(a=170, b=75)])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.a.bitwiseAND(df.b)).collect()</span>
<span class="s2"> [Row((a &amp; b)=10)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_bitwiseXOR_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Compute bitwise XOR of this expression with another expression.</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other</span>
<span class="s2"> a value or :class:`Column` to calculate bitwise xor(^) with</span>
<span class="s2"> this :class:`Column`.</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([Row(a=170, b=75)])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.a.bitwiseXOR(df.b)).collect()</span>
<span class="s2"> [Row((a ^ b)=225)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">bitwiseOR</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;bitwiseOR&quot;</span><span class="p">,</span> <span class="n">_bitwiseOR_doc</span><span class="p">)</span>
<span class="n">bitwiseAND</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;bitwiseAND&quot;</span><span class="p">,</span> <span class="n">_bitwiseAND_doc</span><span class="p">)</span>
<span class="n">bitwiseXOR</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;bitwiseXOR&quot;</span><span class="p">,</span> <span class="n">_bitwiseXOR_doc</span><span class="p">)</span>
<div class="viewcode-block" id="Column.getItem"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.getItem.html#pyspark.sql.Column.getItem">[docs]</a> <span class="k">def</span> <span class="nf">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="sd">&quot;&quot;&quot;</span>
<span class="sd"> An expression that gets an item at position ``ordinal`` out of a list,</span>
<span class="sd"> or gets an item by key out of a dict.</span>
<span class="sd"> .. versionadded:: 1.3.0</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df = spark.createDataFrame([([1, 2], {&quot;key&quot;: &quot;value&quot;})], [&quot;l&quot;, &quot;d&quot;])</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.l.getItem(0), df.d.getItem(&quot;key&quot;)).show()</span>
<span class="sd"> +----+------+</span>
<span class="sd"> |l[0]|d[key]|</span>
<span class="sd"> +----+------+</span>
<span class="sd"> | 1| value|</span>
<span class="sd"> +----+------+</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">key</span><span class="p">,</span> <span class="n">Column</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;A column as &#39;key&#39; in getItem is deprecated as of Spark 3.0, and will not &quot;</span>
<span class="s2">&quot;be supported in the future release. Use `column[key]` or `column.key` syntax &quot;</span>
<span class="s2">&quot;instead.&quot;</span><span class="p">,</span>
<span class="ne">DeprecationWarning</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">key</span><span class="p">]</span></div>
<div class="viewcode-block" id="Column.getField"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.getField.html#pyspark.sql.Column.getField">[docs]</a> <span class="k">def</span> <span class="nf">getField</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="sd">&quot;&quot;&quot;</span>
<span class="sd"> An expression that gets a field by name in a StructField.</span>
<span class="sd"> .. versionadded:: 1.3.0</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="sd"> &gt;&gt;&gt; df = spark.createDataFrame([Row(r=Row(a=1, b=&quot;b&quot;))])</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.r.getField(&quot;b&quot;)).show()</span>
<span class="sd"> +---+</span>
<span class="sd"> |r.b|</span>
<span class="sd"> +---+</span>
<span class="sd"> | b|</span>
<span class="sd"> +---+</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.r.a).show()</span>
<span class="sd"> +---+</span>
<span class="sd"> |r.a|</span>
<span class="sd"> +---+</span>
<span class="sd"> | 1|</span>
<span class="sd"> +---+</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">name</span><span class="p">,</span> <span class="n">Column</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;A column as &#39;name&#39; in getField is deprecated as of Spark 3.0, and will not &quot;</span>
<span class="s2">&quot;be supported in the future release. Use `column[name]` or `column.name` syntax &quot;</span>
<span class="s2">&quot;instead.&quot;</span><span class="p">,</span>
<span class="ne">DeprecationWarning</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">name</span><span class="p">]</span></div>
<div class="viewcode-block" id="Column.withField"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.withField.html#pyspark.sql.Column.withField">[docs]</a> <span class="k">def</span> <span class="nf">withField</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">fieldName</span><span class="p">,</span> <span class="n">col</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> An expression that adds/replaces a field in :class:`StructType` by name.</span>
<span class="sd"> .. versionadded:: 3.1.0</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql.functions import lit</span>
<span class="sd"> &gt;&gt;&gt; df = spark.createDataFrame([Row(a=Row(b=1, c=2))])</span>
<span class="sd"> &gt;&gt;&gt; df.withColumn(&#39;a&#39;, df[&#39;a&#39;].withField(&#39;b&#39;, lit(3))).select(&#39;a.b&#39;).show()</span>
<span class="sd"> +---+</span>
<span class="sd"> | b|</span>
<span class="sd"> +---+</span>
<span class="sd"> | 3|</span>
<span class="sd"> +---+</span>
<span class="sd"> &gt;&gt;&gt; df.withColumn(&#39;a&#39;, df[&#39;a&#39;].withField(&#39;d&#39;, lit(4))).select(&#39;a.d&#39;).show()</span>
<span class="sd"> +---+</span>
<span class="sd"> | d|</span>
<span class="sd"> +---+</span>
<span class="sd"> | 4|</span>
<span class="sd"> +---+</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">fieldName</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;fieldName should be a string&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">col</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;col should be a Column&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">withField</span><span class="p">(</span><span class="n">fieldName</span><span class="p">,</span> <span class="n">col</span><span class="o">.</span><span class="n">_jc</span><span class="p">))</span></div>
<div class="viewcode-block" id="Column.dropFields"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.dropFields.html#pyspark.sql.Column.dropFields">[docs]</a> <span class="k">def</span> <span class="nf">dropFields</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">fieldNames</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> An expression that drops fields in :class:`StructType` by name.</span>
<span class="sd"> .. versionadded:: 3.1.0</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql.functions import col, lit</span>
<span class="sd"> &gt;&gt;&gt; df = spark.createDataFrame([</span>
<span class="sd"> ... Row(a=Row(b=1, c=2, d=3, e=Row(f=4, g=5, h=6)))])</span>
<span class="sd"> &gt;&gt;&gt; df.withColumn(&#39;a&#39;, df[&#39;a&#39;].dropFields(&#39;b&#39;)).show()</span>
<span class="sd"> +-----------------+</span>
<span class="sd"> | a|</span>
<span class="sd"> +-----------------+</span>
<span class="sd"> |{2, 3, {4, 5, 6}}|</span>
<span class="sd"> +-----------------+</span>
<span class="sd"> &gt;&gt;&gt; df.withColumn(&#39;a&#39;, df[&#39;a&#39;].dropFields(&#39;b&#39;, &#39;c&#39;)).show()</span>
<span class="sd"> +--------------+</span>
<span class="sd"> | a|</span>
<span class="sd"> +--------------+</span>
<span class="sd"> |{3, {4, 5, 6}}|</span>
<span class="sd"> +--------------+</span>
<span class="sd"> This method supports dropping multiple nested fields directly e.g.</span>
<span class="sd"> &gt;&gt;&gt; df.withColumn(&quot;a&quot;, col(&quot;a&quot;).dropFields(&quot;e.g&quot;, &quot;e.h&quot;)).show()</span>
<span class="sd"> +--------------+</span>
<span class="sd"> | a|</span>
<span class="sd"> +--------------+</span>
<span class="sd"> |{1, 2, 3, {4}}|</span>
<span class="sd"> +--------------+</span>
<span class="sd"> However, if you are going to add/replace multiple nested fields,</span>
<span class="sd"> it is preferred to extract out the nested struct before</span>
<span class="sd"> adding/replacing multiple fields e.g.</span>
<span class="sd"> &gt;&gt;&gt; df.select(col(&quot;a&quot;).withField(</span>
<span class="sd"> ... &quot;e&quot;, col(&quot;a.e&quot;).dropFields(&quot;g&quot;, &quot;h&quot;)).alias(&quot;a&quot;)</span>
<span class="sd"> ... ).show()</span>
<span class="sd"> +--------------+</span>
<span class="sd"> | a|</span>
<span class="sd"> +--------------+</span>
<span class="sd"> |{1, 2, 3, {4}}|</span>
<span class="sd"> +--------------+</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">sc</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="o">.</span><span class="n">_active_spark_context</span>
<span class="n">jc</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">dropFields</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">fieldNames</span><span class="p">))</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span></div>
<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">item</span><span class="p">):</span>
<span class="k">if</span> <span class="n">item</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">item</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="p">[</span><span class="n">item</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">k</span><span class="p">):</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">slice</span><span class="p">):</span>
<span class="k">if</span> <span class="n">k</span><span class="o">.</span><span class="n">step</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;slice with step is not supported.&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">substr</span><span class="p">(</span><span class="n">k</span><span class="o">.</span><span class="n">start</span><span class="p">,</span> <span class="n">k</span><span class="o">.</span><span class="n">stop</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;apply&quot;</span><span class="p">)(</span><span class="bp">self</span><span class="p">,</span> <span class="n">k</span><span class="p">)</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="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Column is not iterable&quot;</span><span class="p">)</span>
<span class="c1"># string methods</span>
<span class="n">_contains_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Contains the other element. Returns a boolean :class:`Column` based on a string match.</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other</span>
<span class="s2"> string in line. A value as a literal or a :class:`Column`.</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.name.contains(&#39;o&#39;)).collect()</span>
<span class="s2"> [Row(age=5, name=&#39;Bob&#39;)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_rlike_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> SQL RLIKE expression (LIKE with Regex). Returns a boolean :class:`Column` based on a regex</span>
<span class="s2"> match.</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other : str</span>
<span class="s2"> an extended regex expression</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.name.rlike(&#39;ice$&#39;)).collect()</span>
<span class="s2"> [Row(age=2, name=&#39;Alice&#39;)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_like_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> SQL like expression. Returns a boolean :class:`Column` based on a SQL LIKE match.</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other : str</span>
<span class="s2"> a SQL LIKE pattern</span>
<span class="s2"> See Also</span>
<span class="s2"> --------</span>
<span class="s2"> pyspark.sql.Column.rlike</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.name.like(&#39;Al%&#39;)).collect()</span>
<span class="s2"> [Row(age=2, name=&#39;Alice&#39;)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_startswith_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> String starts with. Returns a boolean :class:`Column` based on a string match.</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other : :class:`Column` or str</span>
<span class="s2"> string at start of line (do not use a regex `^`)</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.name.startswith(&#39;Al&#39;)).collect()</span>
<span class="s2"> [Row(age=2, name=&#39;Alice&#39;)]</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.name.startswith(&#39;^Al&#39;)).collect()</span>
<span class="s2"> []</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_endswith_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> String ends with. Returns a boolean :class:`Column` based on a string match.</span>
<span class="s2"> Parameters</span>
<span class="s2"> ----------</span>
<span class="s2"> other : :class:`Column` or str</span>
<span class="s2"> string at end of line (do not use a regex `$`)</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.name.endswith(&#39;ice&#39;)).collect()</span>
<span class="s2"> [Row(age=2, name=&#39;Alice&#39;)]</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.name.endswith(&#39;ice$&#39;)).collect()</span>
<span class="s2"> []</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">contains</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;contains&quot;</span><span class="p">,</span> <span class="n">_contains_doc</span><span class="p">)</span>
<span class="n">rlike</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;rlike&quot;</span><span class="p">,</span> <span class="n">_rlike_doc</span><span class="p">)</span>
<span class="n">like</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;like&quot;</span><span class="p">,</span> <span class="n">_like_doc</span><span class="p">)</span>
<span class="n">startswith</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;startsWith&quot;</span><span class="p">,</span> <span class="n">_startswith_doc</span><span class="p">)</span>
<span class="n">endswith</span> <span class="o">=</span> <span class="n">_bin_op</span><span class="p">(</span><span class="s2">&quot;endsWith&quot;</span><span class="p">,</span> <span class="n">_endswith_doc</span><span class="p">)</span>
<div class="viewcode-block" id="Column.substr"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.substr.html#pyspark.sql.Column.substr">[docs]</a> <span class="k">def</span> <span class="nf">substr</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">startPos</span><span class="p">,</span> <span class="n">length</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a :class:`Column` which is a substring of the column.</span>
<span class="sd"> .. versionadded:: 1.3.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> startPos : :class:`Column` or int</span>
<span class="sd"> start position</span>
<span class="sd"> length : :class:`Column` or int</span>
<span class="sd"> length of the substring</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.name.substr(1, 3).alias(&quot;col&quot;)).collect()</span>
<span class="sd"> [Row(col=&#39;Ali&#39;), Row(col=&#39;Bob&#39;)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">type</span><span class="p">(</span><span class="n">startPos</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">type</span><span class="p">(</span><span class="n">length</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span>
<span class="s2">&quot;startPos and length must be the same type. &quot;</span>
<span class="s2">&quot;Got </span><span class="si">{startPos_t}</span><span class="s2"> and </span><span class="si">{length_t}</span><span class="s2">, respectively.&quot;</span>
<span class="o">.</span><span class="n">format</span><span class="p">(</span>
<span class="n">startPos_t</span><span class="o">=</span><span class="nb">type</span><span class="p">(</span><span class="n">startPos</span><span class="p">),</span>
<span class="n">length_t</span><span class="o">=</span><span class="nb">type</span><span class="p">(</span><span class="n">length</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">startPos</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="n">jc</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">substr</span><span class="p">(</span><span class="n">startPos</span><span class="p">,</span> <span class="n">length</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">startPos</span><span class="p">,</span> <span class="n">Column</span><span class="p">):</span>
<span class="n">jc</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">substr</span><span class="p">(</span><span class="n">startPos</span><span class="o">.</span><span class="n">_jc</span><span class="p">,</span> <span class="n">length</span><span class="o">.</span><span class="n">_jc</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;Unexpected 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">startPos</span><span class="p">))</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span></div>
<div class="viewcode-block" id="Column.isin"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.isin.html#pyspark.sql.Column.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="o">*</span><span class="n">cols</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> A boolean expression that is evaluated to true if the value of this</span>
<span class="sd"> expression is contained by the evaluated values of the arguments.</span>
<span class="sd"> .. versionadded:: 1.5.0</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df[df.name.isin(&quot;Bob&quot;, &quot;Mike&quot;)].collect()</span>
<span class="sd"> [Row(age=5, name=&#39;Bob&#39;)]</span>
<span class="sd"> &gt;&gt;&gt; df[df.age.isin([1, 2, 3])].collect()</span>
<span class="sd"> [Row(age=2, name=&#39;Alice&#39;)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">cols</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">cols</span><span class="p">[</span><span class="mi">0</span><span class="p">],</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">cols</span> <span class="o">=</span> <span class="n">cols</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="n">cols</span> <span class="o">=</span> <span class="p">[</span><span class="n">c</span><span class="o">.</span><span class="n">_jc</span> <span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">c</span><span class="p">,</span> <span class="n">Column</span><span class="p">)</span> <span class="k">else</span> <span class="n">_create_column_from_literal</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">cols</span><span class="p">]</span>
<span class="n">sc</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="o">.</span><span class="n">_active_spark_context</span>
<span class="n">jc</span> <span class="o">=</span> <span class="nb">getattr</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="p">,</span> <span class="s2">&quot;isin&quot;</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">cols</span><span class="p">))</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span></div>
<span class="c1"># order</span>
<span class="n">_asc_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Returns a sort expression based on ascending order of the column.</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([(&#39;Tom&#39;, 80), (&#39;Alice&#39;, None)], [&quot;name&quot;, &quot;height&quot;])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.name).orderBy(df.name.asc()).collect()</span>
<span class="s2"> [Row(name=&#39;Alice&#39;), Row(name=&#39;Tom&#39;)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_asc_nulls_first_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Returns a sort expression based on ascending order of the column, and null values</span>
<span class="s2"> return before non-null values.</span>
<span class="s2"> .. versionadded:: 2.4.0</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([(&#39;Tom&#39;, 80), (None, 60), (&#39;Alice&#39;, None)], [&quot;name&quot;, &quot;height&quot;])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.name).orderBy(df.name.asc_nulls_first()).collect()</span>
<span class="s2"> [Row(name=None), Row(name=&#39;Alice&#39;), Row(name=&#39;Tom&#39;)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_asc_nulls_last_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Returns a sort expression based on ascending order of the column, and null values</span>
<span class="s2"> appear after non-null values.</span>
<span class="s2"> .. versionadded:: 2.4.0</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([(&#39;Tom&#39;, 80), (None, 60), (&#39;Alice&#39;, None)], [&quot;name&quot;, &quot;height&quot;])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.name).orderBy(df.name.asc_nulls_last()).collect()</span>
<span class="s2"> [Row(name=&#39;Alice&#39;), Row(name=&#39;Tom&#39;), Row(name=None)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_desc_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Returns a sort expression based on the descending order of the column.</span>
<span class="s2"> .. versionadded:: 2.4.0</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([(&#39;Tom&#39;, 80), (&#39;Alice&#39;, None)], [&quot;name&quot;, &quot;height&quot;])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.name).orderBy(df.name.desc()).collect()</span>
<span class="s2"> [Row(name=&#39;Tom&#39;), Row(name=&#39;Alice&#39;)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_desc_nulls_first_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Returns a sort expression based on the descending order of the column, and null values</span>
<span class="s2"> appear before non-null values.</span>
<span class="s2"> .. versionadded:: 2.4.0</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([(&#39;Tom&#39;, 80), (None, 60), (&#39;Alice&#39;, None)], [&quot;name&quot;, &quot;height&quot;])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.name).orderBy(df.name.desc_nulls_first()).collect()</span>
<span class="s2"> [Row(name=None), Row(name=&#39;Tom&#39;), Row(name=&#39;Alice&#39;)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_desc_nulls_last_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> Returns a sort expression based on the descending order of the column, and null values</span>
<span class="s2"> appear after non-null values.</span>
<span class="s2"> .. versionadded:: 2.4.0</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([(&#39;Tom&#39;, 80), (None, 60), (&#39;Alice&#39;, None)], [&quot;name&quot;, &quot;height&quot;])</span>
<span class="s2"> &gt;&gt;&gt; df.select(df.name).orderBy(df.name.desc_nulls_last()).collect()</span>
<span class="s2"> [Row(name=&#39;Tom&#39;), Row(name=&#39;Alice&#39;), Row(name=None)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">asc</span> <span class="o">=</span> <span class="n">_unary_op</span><span class="p">(</span><span class="s2">&quot;asc&quot;</span><span class="p">,</span> <span class="n">_asc_doc</span><span class="p">)</span>
<span class="n">asc_nulls_first</span> <span class="o">=</span> <span class="n">_unary_op</span><span class="p">(</span><span class="s2">&quot;asc_nulls_first&quot;</span><span class="p">,</span> <span class="n">_asc_nulls_first_doc</span><span class="p">)</span>
<span class="n">asc_nulls_last</span> <span class="o">=</span> <span class="n">_unary_op</span><span class="p">(</span><span class="s2">&quot;asc_nulls_last&quot;</span><span class="p">,</span> <span class="n">_asc_nulls_last_doc</span><span class="p">)</span>
<span class="n">desc</span> <span class="o">=</span> <span class="n">_unary_op</span><span class="p">(</span><span class="s2">&quot;desc&quot;</span><span class="p">,</span> <span class="n">_desc_doc</span><span class="p">)</span>
<span class="n">desc_nulls_first</span> <span class="o">=</span> <span class="n">_unary_op</span><span class="p">(</span><span class="s2">&quot;desc_nulls_first&quot;</span><span class="p">,</span> <span class="n">_desc_nulls_first_doc</span><span class="p">)</span>
<span class="n">desc_nulls_last</span> <span class="o">=</span> <span class="n">_unary_op</span><span class="p">(</span><span class="s2">&quot;desc_nulls_last&quot;</span><span class="p">,</span> <span class="n">_desc_nulls_last_doc</span><span class="p">)</span>
<span class="n">_isNull_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> True if the current expression is null.</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([Row(name=&#39;Tom&#39;, height=80), Row(name=&#39;Alice&#39;, height=None)])</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.height.isNull()).collect()</span>
<span class="s2"> [Row(name=&#39;Alice&#39;, height=None)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">_isNotNull_doc</span> <span class="o">=</span> <span class="s2">&quot;&quot;&quot;</span>
<span class="s2"> True if the current expression is NOT null.</span>
<span class="s2"> Examples</span>
<span class="s2"> --------</span>
<span class="s2"> &gt;&gt;&gt; from pyspark.sql import Row</span>
<span class="s2"> &gt;&gt;&gt; df = spark.createDataFrame([Row(name=&#39;Tom&#39;, height=80), Row(name=&#39;Alice&#39;, height=None)])</span>
<span class="s2"> &gt;&gt;&gt; df.filter(df.height.isNotNull()).collect()</span>
<span class="s2"> [Row(name=&#39;Tom&#39;, height=80)]</span>
<span class="s2"> &quot;&quot;&quot;</span>
<span class="n">isNull</span> <span class="o">=</span> <span class="n">_unary_op</span><span class="p">(</span><span class="s2">&quot;isNull&quot;</span><span class="p">,</span> <span class="n">_isNull_doc</span><span class="p">)</span>
<span class="n">isNotNull</span> <span class="o">=</span> <span class="n">_unary_op</span><span class="p">(</span><span class="s2">&quot;isNotNull&quot;</span><span class="p">,</span> <span class="n">_isNotNull_doc</span><span class="p">)</span>
<div class="viewcode-block" id="Column.alias"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.alias.html#pyspark.sql.Column.alias">[docs]</a> <span class="k">def</span> <span class="nf">alias</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">alias</span><span class="p">,</span> <span class="o">**</span><span class="n">kwargs</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns this column aliased with a new name or names (in the case of expressions that</span>
<span class="sd"> return more than one column, such as explode).</span>
<span class="sd"> .. versionadded:: 1.3.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> alias : str</span>
<span class="sd"> desired column names (collects all positional arguments passed)</span>
<span class="sd"> Other Parameters</span>
<span class="sd"> ----------------</span>
<span class="sd"> metadata: dict</span>
<span class="sd"> a dict of information to be stored in ``metadata`` attribute of the</span>
<span class="sd"> corresponding :class:`StructField &lt;pyspark.sql.types.StructField&gt;` (optional, keyword</span>
<span class="sd"> only argument)</span>
<span class="sd"> .. versionchanged:: 2.2.0</span>
<span class="sd"> Added optional ``metadata`` argument.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.age.alias(&quot;age2&quot;)).collect()</span>
<span class="sd"> [Row(age2=2), Row(age2=5)]</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.age.alias(&quot;age3&quot;, metadata={&#39;max&#39;: 99})).schema[&#39;age3&#39;].metadata[&#39;max&#39;]</span>
<span class="sd"> 99</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">metadata</span> <span class="o">=</span> <span class="n">kwargs</span><span class="o">.</span><span class="n">pop</span><span class="p">(</span><span class="s1">&#39;metadata&#39;</span><span class="p">,</span> <span class="kc">None</span><span class="p">)</span>
<span class="k">assert</span> <span class="ow">not</span> <span class="n">kwargs</span><span class="p">,</span> <span class="s1">&#39;Unexpected kwargs where passed: </span><span class="si">%s</span><span class="s1">&#39;</span> <span class="o">%</span> <span class="n">kwargs</span>
<span class="n">sc</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="o">.</span><span class="n">_active_spark_context</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">alias</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="n">metadata</span><span class="p">:</span>
<span class="n">jmeta</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">types</span><span class="o">.</span><span class="n">Metadata</span><span class="o">.</span><span class="n">fromJson</span><span class="p">(</span>
<span class="n">json</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">metadata</span><span class="p">))</span>
<span class="k">return</span> <span class="n">Column</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">_jc</span><span class="p">,</span> <span class="s2">&quot;as&quot;</span><span class="p">)(</span><span class="n">alias</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">jmeta</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">Column</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">_jc</span><span class="p">,</span> <span class="s2">&quot;as&quot;</span><span class="p">)(</span><span class="n">alias</span><span class="p">[</span><span class="mi">0</span><span class="p">]))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="n">metadata</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s1">&#39;metadata can only be provided for a single column&#39;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</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">_jc</span><span class="p">,</span> <span class="s2">&quot;as&quot;</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="nb">list</span><span class="p">(</span><span class="n">alias</span><span class="p">))))</span></div>
<span class="n">name</span> <span class="o">=</span> <span class="n">copy_func</span><span class="p">(</span><span class="n">alias</span><span class="p">,</span> <span class="n">sinceversion</span><span class="o">=</span><span class="mf">2.0</span><span class="p">,</span> <span class="n">doc</span><span class="o">=</span><span class="s2">&quot;:func:`name` is an alias for :func:`alias`.&quot;</span><span class="p">)</span>
<div class="viewcode-block" id="Column.cast"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.cast.html#pyspark.sql.Column.cast">[docs]</a> <span class="k">def</span> <span class="nf">cast</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">dataType</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot; Convert the column into type ``dataType``.</span>
<span class="sd"> .. versionadded:: 1.3.0</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.age.cast(&quot;string&quot;).alias(&#39;ages&#39;)).collect()</span>
<span class="sd"> [Row(ages=&#39;2&#39;), Row(ages=&#39;5&#39;)]</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.age.cast(StringType()).alias(&#39;ages&#39;)).collect()</span>
<span class="sd"> [Row(ages=&#39;2&#39;), Row(ages=&#39;5&#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">dataType</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span>
<span class="n">jc</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">dataType</span><span class="p">)</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">dataType</span><span class="p">,</span> <span class="n">DataType</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">builder</span><span class="o">.</span><span class="n">getOrCreate</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="n">dataType</span><span class="o">.</span><span class="n">json</span><span class="p">())</span>
<span class="n">jc</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">cast</span><span class="p">(</span><span class="n">jdt</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;unexpected 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">dataType</span><span class="p">))</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span></div>
<span class="n">astype</span> <span class="o">=</span> <span class="n">copy_func</span><span class="p">(</span><span class="n">cast</span><span class="p">,</span> <span class="n">sinceversion</span><span class="o">=</span><span class="mf">1.4</span><span class="p">,</span> <span class="n">doc</span><span class="o">=</span><span class="s2">&quot;:func:`astype` is an alias for :func:`cast`.&quot;</span><span class="p">)</span>
<div class="viewcode-block" id="Column.between"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.between.html#pyspark.sql.Column.between">[docs]</a> <span class="k">def</span> <span class="nf">between</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">lowerBound</span><span class="p">,</span> <span class="n">upperBound</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> A boolean expression that is evaluated to true if the value of this</span>
<span class="sd"> expression is between the given columns.</span>
<span class="sd"> .. versionadded:: 1.3.0</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.name, df.age.between(2, 4)).show()</span>
<span class="sd"> +-----+---------------------------+</span>
<span class="sd"> | name|((age &gt;= 2) AND (age &lt;= 4))|</span>
<span class="sd"> +-----+---------------------------+</span>
<span class="sd"> |Alice| true|</span>
<span class="sd"> | Bob| false|</span>
<span class="sd"> +-----+---------------------------+</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">&gt;=</span> <span class="n">lowerBound</span><span class="p">)</span> <span class="o">&amp;</span> <span class="p">(</span><span class="bp">self</span> <span class="o">&lt;=</span> <span class="n">upperBound</span><span class="p">)</span></div>
<div class="viewcode-block" id="Column.when"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.when.html#pyspark.sql.Column.when">[docs]</a> <span class="k">def</span> <span class="nf">when</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">condition</span><span class="p">,</span> <span class="n">value</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Evaluates a list of conditions and returns one of multiple possible result expressions.</span>
<span class="sd"> If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions.</span>
<span class="sd"> .. versionadded:: 1.4.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> condition : :class:`Column`</span>
<span class="sd"> a boolean :class:`Column` expression.</span>
<span class="sd"> value</span>
<span class="sd"> a literal value, or a :class:`Column` expression.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql import functions as F</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.name, F.when(df.age &gt; 4, 1).when(df.age &lt; 3, -1).otherwise(0)).show()</span>
<span class="sd"> +-----+------------------------------------------------------------+</span>
<span class="sd"> | name|CASE WHEN (age &gt; 4) THEN 1 WHEN (age &lt; 3) THEN -1 ELSE 0 END|</span>
<span class="sd"> +-----+------------------------------------------------------------+</span>
<span class="sd"> |Alice| -1|</span>
<span class="sd"> | Bob| 1|</span>
<span class="sd"> +-----+------------------------------------------------------------+</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> pyspark.sql.functions.when</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">condition</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;condition should be a Column&quot;</span><span class="p">)</span>
<span class="n">v</span> <span class="o">=</span> <span class="n">value</span><span class="o">.</span><span class="n">_jc</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">Column</span><span class="p">)</span> <span class="k">else</span> <span class="n">value</span>
<span class="n">jc</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">when</span><span class="p">(</span><span class="n">condition</span><span class="o">.</span><span class="n">_jc</span><span class="p">,</span> <span class="n">v</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span></div>
<div class="viewcode-block" id="Column.otherwise"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.otherwise.html#pyspark.sql.Column.otherwise">[docs]</a> <span class="k">def</span> <span class="nf">otherwise</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="sd">&quot;&quot;&quot;</span>
<span class="sd"> Evaluates a list of conditions and returns one of multiple possible result expressions.</span>
<span class="sd"> If :func:`Column.otherwise` is not invoked, None is returned for unmatched conditions.</span>
<span class="sd"> .. versionadded:: 1.4.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> value</span>
<span class="sd"> a literal value, or a :class:`Column` expression.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql import functions as F</span>
<span class="sd"> &gt;&gt;&gt; df.select(df.name, F.when(df.age &gt; 3, 1).otherwise(0)).show()</span>
<span class="sd"> +-----+-------------------------------------+</span>
<span class="sd"> | name|CASE WHEN (age &gt; 3) THEN 1 ELSE 0 END|</span>
<span class="sd"> +-----+-------------------------------------+</span>
<span class="sd"> |Alice| 0|</span>
<span class="sd"> | Bob| 1|</span>
<span class="sd"> +-----+-------------------------------------+</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> pyspark.sql.functions.when</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">v</span> <span class="o">=</span> <span class="n">value</span><span class="o">.</span><span class="n">_jc</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">Column</span><span class="p">)</span> <span class="k">else</span> <span class="n">value</span>
<span class="n">jc</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">otherwise</span><span class="p">(</span><span class="n">v</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span></div>
<div class="viewcode-block" id="Column.over"><a class="viewcode-back" href="../../../reference/api/pyspark.sql.Column.over.html#pyspark.sql.Column.over">[docs]</a> <span class="k">def</span> <span class="nf">over</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">window</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Define a windowing column.</span>
<span class="sd"> .. versionadded:: 1.4.0</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> window : :class:`WindowSpec`</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> :class:`Column`</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql import Window</span>
<span class="sd"> &gt;&gt;&gt; window = Window.partitionBy(&quot;name&quot;).orderBy(&quot;age&quot;) \</span>
<span class="sd"> .rowsBetween(Window.unboundedPreceding, Window.currentRow)</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql.functions import rank, min</span>
<span class="sd"> &gt;&gt;&gt; from pyspark.sql.functions import desc</span>
<span class="sd"> &gt;&gt;&gt; df.withColumn(&quot;rank&quot;, rank().over(window)) \</span>
<span class="sd"> .withColumn(&quot;min&quot;, min(&#39;age&#39;).over(window)).sort(desc(&quot;age&quot;)).show()</span>
<span class="sd"> +---+-----+----+---+</span>
<span class="sd"> |age| name|rank|min|</span>
<span class="sd"> +---+-----+----+---+</span>
<span class="sd"> | 5| Bob| 1| 5|</span>
<span class="sd"> | 2|Alice| 1| 2|</span>
<span class="sd"> +---+-----+----+---+</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="kn">from</span> <span class="nn">pyspark.sql.window</span> <span class="kn">import</span> <span class="n">WindowSpec</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">window</span><span class="p">,</span> <span class="n">WindowSpec</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;window should be WindowSpec&quot;</span><span class="p">)</span>
<span class="n">jc</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">over</span><span class="p">(</span><span class="n">window</span><span class="o">.</span><span class="n">_jspec</span><span class="p">)</span>
<span class="k">return</span> <span class="n">Column</span><span class="p">(</span><span class="n">jc</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">__nonzero__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Cannot convert column into bool: please use &#39;&amp;&#39; for &#39;and&#39;, &#39;|&#39; for &#39;or&#39;, &quot;</span>
<span class="s2">&quot;&#39;~&#39; for &#39;not&#39; when building DataFrame boolean expressions.&quot;</span><span class="p">)</span>
<span class="fm">__bool__</span> <span class="o">=</span> <span class="n">__nonzero__</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="k">return</span> <span class="s2">&quot;Column&lt;&#39;</span><span class="si">%s</span><span class="s2">&#39;&gt;&quot;</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jc</span><span class="o">.</span><span class="n">toString</span><span class="p">()</span></div>
<span class="k">def</span> <span class="nf">_test</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.column</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">column</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">&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;sql.column tests&quot;</span><span class="p">)</span>\
<span class="o">.</span><span class="n">getOrCreate</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="n">globs</span><span class="p">[</span><span class="s1">&#39;spark&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">spark</span>
<span class="n">globs</span><span class="p">[</span><span class="s1">&#39;df&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">sc</span><span class="o">.</span><span class="n">parallelize</span><span class="p">([(</span><span class="mi">2</span><span class="p">,</span> <span class="s1">&#39;Alice&#39;</span><span class="p">),</span> <span class="p">(</span><span class="mi">5</span><span class="p">,</span> <span class="s1">&#39;Bob&#39;</span><span class="p">)])</span> \
<span class="o">.</span><span class="n">toDF</span><span class="p">(</span><span class="n">StructType</span><span class="p">([</span><span class="n">StructField</span><span class="p">(</span><span class="s1">&#39;age&#39;</span><span class="p">,</span> <span class="n">IntegerType</span><span class="p">()),</span>
<span class="n">StructField</span><span class="p">(</span><span class="s1">&#39;name&#39;</span><span class="p">,</span> <span class="n">StringType</span><span class="p">())]))</span>
<span class="p">(</span><span class="n">failure_count</span><span class="p">,</span> <span class="n">test_count</span><span class="p">)</span> <span class="o">=</span> <span class="n">doctest</span><span class="o">.</span><span class="n">testmod</span><span class="p">(</span>
<span class="n">pyspark</span><span class="o">.</span><span class="n">sql</span><span class="o">.</span><span class="n">column</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="o">|</span> <span class="n">doctest</span><span class="o">.</span><span class="n">REPORT_NDIFF</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>
</div>
<div class='prev-next-bottom'>
</div>
</main>
</div>
</div>
<script src="../../../_static/js/index.3da636dd464baa7582d2.js"></script>
<footer class="footer mt-5 mt-md-0">
<div class="container">
<p>
&copy; Copyright .<br/>
Created using <a href="http://sphinx-doc.org/">Sphinx</a> 3.0.4.<br/>
</p>
</div>
</footer>
</body>
</html>