| |
| <!DOCTYPE html> |
| |
| <html> |
| <head> |
| <meta charset="utf-8" /> |
| <title>pyspark.sql.group — PySpark 3.3.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="search" title="Search" href="../../../search.html" /> |
| <meta name="viewport" content="width=device-width, initial-scale=1" /> |
| <meta name="docsearch:language" content="en" /> |
| </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.group</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="kn">import</span> <span class="nn">sys</span> |
| |
| <span class="kn">from</span> <span class="nn">typing</span> <span class="kn">import</span> <span class="n">Callable</span><span class="p">,</span> <span class="n">List</span><span class="p">,</span> <span class="n">Optional</span><span class="p">,</span> <span class="n">TYPE_CHECKING</span><span class="p">,</span> <span class="n">overload</span><span class="p">,</span> <span class="n">Dict</span><span class="p">,</span> <span class="n">Union</span><span class="p">,</span> <span class="n">cast</span><span class="p">,</span> <span class="n">Tuple</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.column</span> <span class="kn">import</span> <span class="n">Column</span><span class="p">,</span> <span class="n">_to_seq</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="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.pandas.group_ops</span> <span class="kn">import</span> <span class="n">PandasGroupedOpsMixin</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql.types</span> <span class="kn">import</span> <span class="n">StructType</span><span class="p">,</span> <span class="n">StructField</span><span class="p">,</span> <span class="n">IntegerType</span><span class="p">,</span> <span class="n">StringType</span> |
| |
| <span class="k">if</span> <span class="n">TYPE_CHECKING</span><span class="p">:</span> |
| <span class="kn">from</span> <span class="nn">pyspark.sql._typing</span> <span class="kn">import</span> <span class="n">LiteralType</span> |
| |
| <span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s2">"GroupedData"</span><span class="p">]</span> |
| |
| |
| <span class="k">def</span> <span class="nf">dfapi</span><span class="p">(</span><span class="n">f</span><span class="p">:</span> <span class="n">Callable</span><span class="p">)</span> <span class="o">-></span> <span class="n">Callable</span><span class="p">:</span> |
| <span class="k">def</span> <span class="nf">_api</span><span class="p">(</span><span class="bp">self</span><span class="p">:</span> <span class="s2">"GroupedData"</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="n">name</span> <span class="o">=</span> <span class="n">f</span><span class="o">.</span><span class="vm">__name__</span> |
| <span class="n">jdf</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">_jgd</span><span class="p">,</span> <span class="n">name</span><span class="p">)()</span> |
| <span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">jdf</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">session</span><span class="p">)</span> |
| |
| <span class="n">_api</span><span class="o">.</span><span class="vm">__name__</span> <span class="o">=</span> <span class="n">f</span><span class="o">.</span><span class="vm">__name__</span> |
| <span class="n">_api</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">f</span><span class="o">.</span><span class="vm">__doc__</span> |
| <span class="k">return</span> <span class="n">_api</span> |
| |
| |
| <span class="k">def</span> <span class="nf">df_varargs_api</span><span class="p">(</span><span class="n">f</span><span class="p">:</span> <span class="n">Callable</span><span class="p">)</span> <span class="o">-></span> <span class="n">Callable</span><span class="p">:</span> |
| <span class="k">def</span> <span class="nf">_api</span><span class="p">(</span><span class="bp">self</span><span class="p">:</span> <span class="s2">"GroupedData"</span><span class="p">,</span> <span class="o">*</span><span class="n">cols</span><span class="p">:</span> <span class="nb">str</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="n">name</span> <span class="o">=</span> <span class="n">f</span><span class="o">.</span><span class="vm">__name__</span> |
| <span class="n">jdf</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">_jgd</span><span class="p">,</span> <span class="n">name</span><span class="p">)(</span><span class="n">_to_seq</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">session</span><span class="o">.</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">DataFrame</span><span class="p">(</span><span class="n">jdf</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">session</span><span class="p">)</span> |
| |
| <span class="n">_api</span><span class="o">.</span><span class="vm">__name__</span> <span class="o">=</span> <span class="n">f</span><span class="o">.</span><span class="vm">__name__</span> |
| <span class="n">_api</span><span class="o">.</span><span class="vm">__doc__</span> <span class="o">=</span> <span class="n">f</span><span class="o">.</span><span class="vm">__doc__</span> |
| <span class="k">return</span> <span class="n">_api</span> |
| |
| |
| <div class="viewcode-block" id="GroupedData"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.html#pyspark.sql.GroupedData">[docs]</a><span class="k">class</span> <span class="nc">GroupedData</span><span class="p">(</span><span class="n">PandasGroupedOpsMixin</span><span class="p">):</span> |
| <span class="sd">"""</span> |
| <span class="sd"> A set of methods for aggregations on a :class:`DataFrame`,</span> |
| <span class="sd"> created by :func:`DataFrame.groupBy`.</span> |
| |
| <span class="sd"> .. versionadded:: 1.3</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">jgd</span><span class="p">:</span> <span class="n">JavaObject</span><span class="p">,</span> <span class="n">df</span><span class="p">:</span> <span class="n">DataFrame</span><span class="p">):</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_jgd</span> <span class="o">=</span> <span class="n">jgd</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">_df</span> <span class="o">=</span> <span class="n">df</span> |
| <span class="bp">self</span><span class="o">.</span><span class="n">session</span><span class="p">:</span> <span class="n">SparkSession</span> <span class="o">=</span> <span class="n">df</span><span class="o">.</span><span class="n">sparkSession</span> |
| |
| <span class="nd">@overload</span> |
| <span class="k">def</span> <span class="nf">agg</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">exprs</span><span class="p">:</span> <span class="n">Column</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="o">...</span> |
| |
| <span class="nd">@overload</span> |
| <span class="k">def</span> <span class="nf">agg</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">__exprs</span><span class="p">:</span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">str</span><span class="p">])</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="o">...</span> |
| |
| <div class="viewcode-block" id="GroupedData.agg"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.agg.html#pyspark.sql.GroupedData.agg">[docs]</a> <span class="k">def</span> <span class="nf">agg</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="o">*</span><span class="n">exprs</span><span class="p">:</span> <span class="n">Union</span><span class="p">[</span><span class="n">Column</span><span class="p">,</span> <span class="n">Dict</span><span class="p">[</span><span class="nb">str</span><span class="p">,</span> <span class="nb">str</span><span class="p">]])</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="sd">"""Compute aggregates and returns the result as a :class:`DataFrame`.</span> |
| |
| <span class="sd"> The available aggregate functions can be:</span> |
| |
| <span class="sd"> 1. built-in aggregation functions, such as `avg`, `max`, `min`, `sum`, `count`</span> |
| |
| <span class="sd"> 2. group aggregate pandas UDFs, created with :func:`pyspark.sql.functions.pandas_udf`</span> |
| |
| <span class="sd"> .. note:: There is no partial aggregation with group aggregate UDFs, i.e.,</span> |
| <span class="sd"> a full shuffle is required. Also, all the data of a group will be loaded into</span> |
| <span class="sd"> memory, so the user should be aware of the potential OOM risk if data is skewed</span> |
| <span class="sd"> and certain groups are too large to fit in memory.</span> |
| |
| <span class="sd"> .. seealso:: :func:`pyspark.sql.functions.pandas_udf`</span> |
| |
| <span class="sd"> If ``exprs`` is a single :class:`dict` mapping from string to string, then the key</span> |
| <span class="sd"> is the column to perform aggregation on, and the value is the aggregate function.</span> |
| |
| <span class="sd"> Alternatively, ``exprs`` can also be a list of aggregate :class:`Column` expressions.</span> |
| |
| <span class="sd"> .. versionadded:: 1.3.0</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> exprs : dict</span> |
| <span class="sd"> a dict mapping from column name (string) to aggregate functions (string),</span> |
| <span class="sd"> or a list of :class:`Column`.</span> |
| |
| <span class="sd"> Notes</span> |
| <span class="sd"> -----</span> |
| <span class="sd"> Built-in aggregation functions and group aggregate pandas UDFs cannot be mixed</span> |
| <span class="sd"> in a single call to this function.</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> >>> gdf = df.groupBy(df.name)</span> |
| <span class="sd"> >>> sorted(gdf.agg({"*": "count"}).collect())</span> |
| <span class="sd"> [Row(name='Alice', count(1)=1), Row(name='Bob', count(1)=1)]</span> |
| |
| <span class="sd"> >>> from pyspark.sql import functions as F</span> |
| <span class="sd"> >>> sorted(gdf.agg(F.min(df.age)).collect())</span> |
| <span class="sd"> [Row(name='Alice', min(age)=2), Row(name='Bob', min(age)=5)]</span> |
| |
| <span class="sd"> >>> from pyspark.sql.functions import pandas_udf, PandasUDFType</span> |
| <span class="sd"> >>> @pandas_udf('int', PandasUDFType.GROUPED_AGG) # doctest: +SKIP</span> |
| <span class="sd"> ... def min_udf(v):</span> |
| <span class="sd"> ... return v.min()</span> |
| <span class="sd"> >>> sorted(gdf.agg(min_udf(df.age)).collect()) # doctest: +SKIP</span> |
| <span class="sd"> [Row(name='Alice', min_udf(age)=2), Row(name='Bob', min_udf(age)=5)]</span> |
| <span class="sd"> """</span> |
| <span class="k">assert</span> <span class="n">exprs</span><span class="p">,</span> <span class="s2">"exprs should not be empty"</span> |
| <span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">exprs</span><span class="p">)</span> <span class="o">==</span> <span class="mi">1</span> <span class="ow">and</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">exprs</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="nb">dict</span><span class="p">):</span> |
| <span class="n">jdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jgd</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">exprs</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="c1"># Columns</span> |
| <span class="k">assert</span> <span class="nb">all</span><span class="p">(</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">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">exprs</span><span class="p">),</span> <span class="s2">"all exprs should be Column"</span> |
| <span class="n">exprs</span> <span class="o">=</span> <span class="n">cast</span><span class="p">(</span><span class="n">Tuple</span><span class="p">[</span><span class="n">Column</span><span class="p">,</span> <span class="o">...</span><span class="p">],</span> <span class="n">exprs</span><span class="p">)</span> |
| <span class="n">jdf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jgd</span><span class="o">.</span><span class="n">agg</span><span class="p">(</span><span class="n">exprs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span><span class="o">.</span><span class="n">_jc</span><span class="p">,</span> <span class="n">_to_seq</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">session</span><span class="o">.</span><span class="n">_sc</span><span class="p">,</span> <span class="p">[</span><span class="n">c</span><span class="o">.</span><span class="n">_jc</span> <span class="k">for</span> <span class="n">c</span> <span class="ow">in</span> <span class="n">exprs</span><span class="p">[</span><span class="mi">1</span><span class="p">:]]))</span> |
| <span class="k">return</span> <span class="n">DataFrame</span><span class="p">(</span><span class="n">jdf</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">session</span><span class="p">)</span></div> |
| |
| <div class="viewcode-block" id="GroupedData.count"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.count.html#pyspark.sql.GroupedData.count">[docs]</a> <span class="nd">@dfapi</span> |
| <span class="k">def</span> <span class="nf">count</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="sd">"""Counts the number of records for each group.</span> |
| |
| <span class="sd"> .. versionadded:: 1.3.0</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> >>> sorted(df.groupBy(df.age).count().collect())</span> |
| <span class="sd"> [Row(age=2, count=1), Row(age=5, count=1)]</span> |
| <span class="sd"> """</span></div> |
| |
| <div class="viewcode-block" id="GroupedData.mean"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.mean.html#pyspark.sql.GroupedData.mean">[docs]</a> <span class="nd">@df_varargs_api</span> |
| <span class="k">def</span> <span class="nf">mean</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="nb">str</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="sd">"""Computes average values for each numeric columns for each group.</span> |
| |
| <span class="sd"> :func:`mean` is an alias for :func:`avg`.</span> |
| |
| <span class="sd"> .. versionadded:: 1.3.0</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> cols : str</span> |
| <span class="sd"> column names. Non-numeric columns are ignored.</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> >>> df.groupBy().mean('age').collect()</span> |
| <span class="sd"> [Row(avg(age)=3.5)]</span> |
| <span class="sd"> >>> df3.groupBy().mean('age', 'height').collect()</span> |
| <span class="sd"> [Row(avg(age)=3.5, avg(height)=82.5)]</span> |
| <span class="sd"> """</span></div> |
| |
| <div class="viewcode-block" id="GroupedData.avg"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.avg.html#pyspark.sql.GroupedData.avg">[docs]</a> <span class="nd">@df_varargs_api</span> |
| <span class="k">def</span> <span class="nf">avg</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="nb">str</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="sd">"""Computes average values for each numeric columns for each group.</span> |
| |
| <span class="sd"> :func:`mean` is an alias for :func:`avg`.</span> |
| |
| <span class="sd"> .. versionadded:: 1.3.0</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> cols : str</span> |
| <span class="sd"> column names. Non-numeric columns are ignored.</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> >>> df.groupBy().avg('age').collect()</span> |
| <span class="sd"> [Row(avg(age)=3.5)]</span> |
| <span class="sd"> >>> df3.groupBy().avg('age', 'height').collect()</span> |
| <span class="sd"> [Row(avg(age)=3.5, avg(height)=82.5)]</span> |
| <span class="sd"> """</span></div> |
| |
| <div class="viewcode-block" id="GroupedData.max"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.max.html#pyspark.sql.GroupedData.max">[docs]</a> <span class="nd">@df_varargs_api</span> |
| <span class="k">def</span> <span class="nf">max</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="nb">str</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="sd">"""Computes the max value for each numeric columns for each group.</span> |
| |
| <span class="sd"> .. versionadded:: 1.3.0</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> >>> df.groupBy().max('age').collect()</span> |
| <span class="sd"> [Row(max(age)=5)]</span> |
| <span class="sd"> >>> df3.groupBy().max('age', 'height').collect()</span> |
| <span class="sd"> [Row(max(age)=5, max(height)=85)]</span> |
| <span class="sd"> """</span></div> |
| |
| <div class="viewcode-block" id="GroupedData.min"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.min.html#pyspark.sql.GroupedData.min">[docs]</a> <span class="nd">@df_varargs_api</span> |
| <span class="k">def</span> <span class="nf">min</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="nb">str</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="sd">"""Computes the min value for each numeric column for each group.</span> |
| |
| <span class="sd"> .. versionadded:: 1.3.0</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> cols : str</span> |
| <span class="sd"> column names. Non-numeric columns are ignored.</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> >>> df.groupBy().min('age').collect()</span> |
| <span class="sd"> [Row(min(age)=2)]</span> |
| <span class="sd"> >>> df3.groupBy().min('age', 'height').collect()</span> |
| <span class="sd"> [Row(min(age)=2, min(height)=80)]</span> |
| <span class="sd"> """</span></div> |
| |
| <div class="viewcode-block" id="GroupedData.sum"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.sum.html#pyspark.sql.GroupedData.sum">[docs]</a> <span class="nd">@df_varargs_api</span> |
| <span class="k">def</span> <span class="nf">sum</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="nb">str</span><span class="p">)</span> <span class="o">-></span> <span class="n">DataFrame</span><span class="p">:</span> |
| <span class="sd">"""Computes the sum for each numeric columns for each group.</span> |
| |
| <span class="sd"> .. versionadded:: 1.3.0</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> cols : str</span> |
| <span class="sd"> column names. Non-numeric columns are ignored.</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> >>> df.groupBy().sum('age').collect()</span> |
| <span class="sd"> [Row(sum(age)=7)]</span> |
| <span class="sd"> >>> df3.groupBy().sum('age', 'height').collect()</span> |
| <span class="sd"> [Row(sum(age)=7, sum(height)=165)]</span> |
| <span class="sd"> """</span></div> |
| |
| <div class="viewcode-block" id="GroupedData.pivot"><a class="viewcode-back" href="../../../reference/pyspark.sql/api/pyspark.sql.GroupedData.pivot.html#pyspark.sql.GroupedData.pivot">[docs]</a> <span class="k">def</span> <span class="nf">pivot</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">pivot_col</span><span class="p">:</span> <span class="nb">str</span><span class="p">,</span> <span class="n">values</span><span class="p">:</span> <span class="n">Optional</span><span class="p">[</span><span class="n">List</span><span class="p">[</span><span class="s2">"LiteralType"</span><span class="p">]]</span> <span class="o">=</span> <span class="kc">None</span><span class="p">)</span> <span class="o">-></span> <span class="s2">"GroupedData"</span><span class="p">:</span> |
| <span class="sd">"""</span> |
| <span class="sd"> Pivots a column of the current :class:`DataFrame` and perform the specified aggregation.</span> |
| <span class="sd"> There are two versions of pivot function: one that requires the caller to specify the list</span> |
| <span class="sd"> of distinct values to pivot on, and one that does not. The latter is more concise but less</span> |
| <span class="sd"> efficient, because Spark needs to first compute the list of distinct values internally.</span> |
| |
| <span class="sd"> .. versionadded:: 1.6.0</span> |
| |
| <span class="sd"> Parameters</span> |
| <span class="sd"> ----------</span> |
| <span class="sd"> pivot_col : str</span> |
| <span class="sd"> Name of the column to pivot.</span> |
| <span class="sd"> values : list, optional</span> |
| <span class="sd"> List of values that will be translated to columns in the output DataFrame.</span> |
| |
| <span class="sd"> Examples</span> |
| <span class="sd"> --------</span> |
| <span class="sd"> # Compute the sum of earnings for each year by course with each course as a separate column</span> |
| |
| <span class="sd"> >>> df4.groupBy("year").pivot("course", ["dotNET", "Java"]).sum("earnings").collect()</span> |
| <span class="sd"> [Row(year=2012, dotNET=15000, Java=20000), Row(year=2013, dotNET=48000, Java=30000)]</span> |
| |
| <span class="sd"> # Or without specifying column values (less efficient)</span> |
| |
| <span class="sd"> >>> df4.groupBy("year").pivot("course").sum("earnings").collect()</span> |
| <span class="sd"> [Row(year=2012, Java=20000, dotNET=15000), Row(year=2013, Java=30000, dotNET=48000)]</span> |
| <span class="sd"> >>> df5.groupBy("sales.year").pivot("sales.course").sum("sales.earnings").collect()</span> |
| <span class="sd"> [Row(year=2012, Java=20000, dotNET=15000), Row(year=2013, Java=30000, dotNET=48000)]</span> |
| <span class="sd"> """</span> |
| <span class="k">if</span> <span class="n">values</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span> |
| <span class="n">jgd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jgd</span><span class="o">.</span><span class="n">pivot</span><span class="p">(</span><span class="n">pivot_col</span><span class="p">)</span> |
| <span class="k">else</span><span class="p">:</span> |
| <span class="n">jgd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jgd</span><span class="o">.</span><span class="n">pivot</span><span class="p">(</span><span class="n">pivot_col</span><span class="p">,</span> <span class="n">values</span><span class="p">)</span> |
| <span class="k">return</span> <span class="n">GroupedData</span><span class="p">(</span><span class="n">jgd</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_df</span><span class="p">)</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">Row</span><span class="p">,</span> <span class="n">SparkSession</span> |
| <span class="kn">import</span> <span class="nn">pyspark.sql.group</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">group</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.group tests"</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="s2">"sc"</span><span class="p">]</span> <span class="o">=</span> <span class="n">sc</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="n">globs</span><span class="p">[</span><span class="s2">"df"</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="s2">"Alice"</span><span class="p">),</span> <span class="p">(</span><span class="mi">5</span><span class="p">,</span> <span class="s2">"Bob"</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="s2">"age"</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="s2">"name"</span><span class="p">,</span> <span class="n">StringType</span><span class="p">())])</span> |
| <span class="p">)</span> |
| <span class="n">globs</span><span class="p">[</span><span class="s2">"df3"</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="p">[</span><span class="n">Row</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="s2">"Alice"</span><span class="p">,</span> <span class="n">age</span><span class="o">=</span><span class="mi">2</span><span class="p">,</span> <span class="n">height</span><span class="o">=</span><span class="mi">80</span><span class="p">),</span> <span class="n">Row</span><span class="p">(</span><span class="n">name</span><span class="o">=</span><span class="s2">"Bob"</span><span class="p">,</span> <span class="n">age</span><span class="o">=</span><span class="mi">5</span><span class="p">,</span> <span class="n">height</span><span class="o">=</span><span class="mi">85</span><span class="p">)]</span> |
| <span class="p">)</span><span class="o">.</span><span class="n">toDF</span><span class="p">()</span> |
| <span class="n">globs</span><span class="p">[</span><span class="s2">"df4"</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="p">[</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"dotNET"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2012</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">10000</span><span class="p">),</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"Java"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2012</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">20000</span><span class="p">),</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"dotNET"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2012</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">5000</span><span class="p">),</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"dotNET"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2013</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">48000</span><span class="p">),</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"Java"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2013</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">30000</span><span class="p">),</span> |
| <span class="p">]</span> |
| <span class="p">)</span><span class="o">.</span><span class="n">toDF</span><span class="p">()</span> |
| <span class="n">globs</span><span class="p">[</span><span class="s2">"df5"</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="p">[</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">training</span><span class="o">=</span><span class="s2">"expert"</span><span class="p">,</span> <span class="n">sales</span><span class="o">=</span><span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"dotNET"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2012</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">10000</span><span class="p">)),</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">training</span><span class="o">=</span><span class="s2">"junior"</span><span class="p">,</span> <span class="n">sales</span><span class="o">=</span><span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"Java"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2012</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">20000</span><span class="p">)),</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">training</span><span class="o">=</span><span class="s2">"expert"</span><span class="p">,</span> <span class="n">sales</span><span class="o">=</span><span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"dotNET"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2012</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">5000</span><span class="p">)),</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">training</span><span class="o">=</span><span class="s2">"junior"</span><span class="p">,</span> <span class="n">sales</span><span class="o">=</span><span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"dotNET"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2013</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">48000</span><span class="p">)),</span> |
| <span class="n">Row</span><span class="p">(</span><span class="n">training</span><span class="o">=</span><span class="s2">"expert"</span><span class="p">,</span> <span class="n">sales</span><span class="o">=</span><span class="n">Row</span><span class="p">(</span><span class="n">course</span><span class="o">=</span><span class="s2">"Java"</span><span class="p">,</span> <span class="n">year</span><span class="o">=</span><span class="mi">2013</span><span class="p">,</span> <span class="n">earnings</span><span class="o">=</span><span class="mi">30000</span><span class="p">)),</span> |
| <span class="p">]</span> |
| <span class="p">)</span><span class="o">.</span><span class="n">toDF</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">group</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="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> |
| |
| </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> |
| © Copyright .<br/> |
| Created using <a href="http://sphinx-doc.org/">Sphinx</a> 3.0.4.<br/> |
| </p> |
| </div> |
| </footer> |
| </body> |
| </html> |