blob: b3bafff66e501cdcbf33d36e6151e3bda310da91 [file] [log] [blame]
<!DOCTYPE html>
<html>
<head>
<meta charset="utf-8" />
<title>pyspark.rdd &#8212; PySpark 3.2.2 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.rdd</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">copy</span>
<span class="kn">import</span> <span class="nn">sys</span>
<span class="kn">import</span> <span class="nn">os</span>
<span class="kn">import</span> <span class="nn">operator</span>
<span class="kn">import</span> <span class="nn">shlex</span>
<span class="kn">import</span> <span class="nn">warnings</span>
<span class="kn">import</span> <span class="nn">heapq</span>
<span class="kn">import</span> <span class="nn">bisect</span>
<span class="kn">import</span> <span class="nn">random</span>
<span class="kn">from</span> <span class="nn">subprocess</span> <span class="kn">import</span> <span class="n">Popen</span><span class="p">,</span> <span class="n">PIPE</span>
<span class="kn">from</span> <span class="nn">threading</span> <span class="kn">import</span> <span class="n">Thread</span>
<span class="kn">from</span> <span class="nn">collections</span> <span class="kn">import</span> <span class="n">defaultdict</span>
<span class="kn">from</span> <span class="nn">itertools</span> <span class="kn">import</span> <span class="n">chain</span>
<span class="kn">from</span> <span class="nn">functools</span> <span class="kn">import</span> <span class="n">reduce</span>
<span class="kn">from</span> <span class="nn">math</span> <span class="kn">import</span> <span class="n">sqrt</span><span class="p">,</span> <span class="n">log</span><span class="p">,</span> <span class="n">isinf</span><span class="p">,</span> <span class="n">isnan</span><span class="p">,</span> <span class="nb">pow</span><span class="p">,</span> <span class="n">ceil</span>
<span class="kn">from</span> <span class="nn">pyspark.java_gateway</span> <span class="kn">import</span> <span class="n">local_connect_and_auth</span>
<span class="kn">from</span> <span class="nn">pyspark.serializers</span> <span class="kn">import</span> <span class="n">AutoBatchedSerializer</span><span class="p">,</span> <span class="n">BatchedSerializer</span><span class="p">,</span> <span class="n">NoOpSerializer</span><span class="p">,</span> \
<span class="n">CartesianDeserializer</span><span class="p">,</span> <span class="n">CloudPickleSerializer</span><span class="p">,</span> <span class="n">PairDeserializer</span><span class="p">,</span> <span class="n">PickleSerializer</span><span class="p">,</span> \
<span class="n">pack_long</span><span class="p">,</span> <span class="n">read_int</span><span class="p">,</span> <span class="n">write_int</span>
<span class="kn">from</span> <span class="nn">pyspark.join</span> <span class="kn">import</span> <span class="n">python_join</span><span class="p">,</span> <span class="n">python_left_outer_join</span><span class="p">,</span> \
<span class="n">python_right_outer_join</span><span class="p">,</span> <span class="n">python_full_outer_join</span><span class="p">,</span> <span class="n">python_cogroup</span>
<span class="kn">from</span> <span class="nn">pyspark.statcounter</span> <span class="kn">import</span> <span class="n">StatCounter</span>
<span class="kn">from</span> <span class="nn">pyspark.rddsampler</span> <span class="kn">import</span> <span class="n">RDDSampler</span><span class="p">,</span> <span class="n">RDDRangeSampler</span><span class="p">,</span> <span class="n">RDDStratifiedSampler</span>
<span class="kn">from</span> <span class="nn">pyspark.storagelevel</span> <span class="kn">import</span> <span class="n">StorageLevel</span>
<span class="kn">from</span> <span class="nn">pyspark.resource.requests</span> <span class="kn">import</span> <span class="n">ExecutorResourceRequests</span><span class="p">,</span> <span class="n">TaskResourceRequests</span>
<span class="kn">from</span> <span class="nn">pyspark.resource.profile</span> <span class="kn">import</span> <span class="n">ResourceProfile</span>
<span class="kn">from</span> <span class="nn">pyspark.resultiterable</span> <span class="kn">import</span> <span class="n">ResultIterable</span>
<span class="kn">from</span> <span class="nn">pyspark.shuffle</span> <span class="kn">import</span> <span class="n">Aggregator</span><span class="p">,</span> <span class="n">ExternalMerger</span><span class="p">,</span> \
<span class="n">get_used_memory</span><span class="p">,</span> <span class="n">ExternalSorter</span><span class="p">,</span> <span class="n">ExternalGroupBy</span>
<span class="kn">from</span> <span class="nn">pyspark.traceback_utils</span> <span class="kn">import</span> <span class="n">SCCallSiteSync</span>
<span class="kn">from</span> <span class="nn">pyspark.util</span> <span class="kn">import</span> <span class="n">fail_on_stopiteration</span><span class="p">,</span> <span class="n">_parse_memory</span>
<span class="n">__all__</span> <span class="o">=</span> <span class="p">[</span><span class="s2">&quot;RDD&quot;</span><span class="p">]</span>
<span class="k">class</span> <span class="nc">PythonEvalType</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Evaluation type of python rdd.</span>
<span class="sd"> These values are internal to PySpark.</span>
<span class="sd"> These values should match values in org.apache.spark.api.python.PythonEvalType.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">NON_UDF</span> <span class="o">=</span> <span class="mi">0</span>
<span class="n">SQL_BATCHED_UDF</span> <span class="o">=</span> <span class="mi">100</span>
<span class="n">SQL_SCALAR_PANDAS_UDF</span> <span class="o">=</span> <span class="mi">200</span>
<span class="n">SQL_GROUPED_MAP_PANDAS_UDF</span> <span class="o">=</span> <span class="mi">201</span>
<span class="n">SQL_GROUPED_AGG_PANDAS_UDF</span> <span class="o">=</span> <span class="mi">202</span>
<span class="n">SQL_WINDOW_AGG_PANDAS_UDF</span> <span class="o">=</span> <span class="mi">203</span>
<span class="n">SQL_SCALAR_PANDAS_ITER_UDF</span> <span class="o">=</span> <span class="mi">204</span>
<span class="n">SQL_MAP_PANDAS_ITER_UDF</span> <span class="o">=</span> <span class="mi">205</span>
<span class="n">SQL_COGROUPED_MAP_PANDAS_UDF</span> <span class="o">=</span> <span class="mi">206</span>
<span class="k">def</span> <span class="nf">portable_hash</span><span class="p">(</span><span class="n">x</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> This function returns consistent hash code for builtin types, especially</span>
<span class="sd"> for None and tuple with None.</span>
<span class="sd"> The algorithm is similar to that one used by CPython 2.7</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; portable_hash(None)</span>
<span class="sd"> 0</span>
<span class="sd"> &gt;&gt;&gt; portable_hash((None, 1)) &amp; 0xffffffff</span>
<span class="sd"> 219750521</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="s1">&#39;PYTHONHASHSEED&#39;</span> <span class="ow">not</span> <span class="ow">in</span> <span class="n">os</span><span class="o">.</span><span class="n">environ</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span><span class="s2">&quot;Randomness of hash of string should be disabled via PYTHONHASHSEED&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">x</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="mi">0</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">):</span>
<span class="n">h</span> <span class="o">=</span> <span class="mh">0x345678</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">x</span><span class="p">:</span>
<span class="n">h</span> <span class="o">^=</span> <span class="n">portable_hash</span><span class="p">(</span><span class="n">i</span><span class="p">)</span>
<span class="n">h</span> <span class="o">*=</span> <span class="mi">1000003</span>
<span class="n">h</span> <span class="o">&amp;=</span> <span class="n">sys</span><span class="o">.</span><span class="n">maxsize</span>
<span class="n">h</span> <span class="o">^=</span> <span class="nb">len</span><span class="p">(</span><span class="n">x</span><span class="p">)</span>
<span class="k">if</span> <span class="n">h</span> <span class="o">==</span> <span class="o">-</span><span class="mi">1</span><span class="p">:</span>
<span class="n">h</span> <span class="o">=</span> <span class="o">-</span><span class="mi">2</span>
<span class="k">return</span> <span class="nb">int</span><span class="p">(</span><span class="n">h</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">hash</span><span class="p">(</span><span class="n">x</span><span class="p">)</span>
<span class="k">class</span> <span class="nc">BoundedFloat</span><span class="p">(</span><span class="nb">float</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Bounded value is generated by approximate job, with confidence and low</span>
<span class="sd"> bound and high bound.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; BoundedFloat(100.0, 0.95, 95.0, 105.0)</span>
<span class="sd"> 100.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="fm">__new__</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">mean</span><span class="p">,</span> <span class="n">confidence</span><span class="p">,</span> <span class="n">low</span><span class="p">,</span> <span class="n">high</span><span class="p">):</span>
<span class="n">obj</span> <span class="o">=</span> <span class="nb">float</span><span class="o">.</span><span class="fm">__new__</span><span class="p">(</span><span class="bp">cls</span><span class="p">,</span> <span class="n">mean</span><span class="p">)</span>
<span class="n">obj</span><span class="o">.</span><span class="n">confidence</span> <span class="o">=</span> <span class="n">confidence</span>
<span class="n">obj</span><span class="o">.</span><span class="n">low</span> <span class="o">=</span> <span class="n">low</span>
<span class="n">obj</span><span class="o">.</span><span class="n">high</span> <span class="o">=</span> <span class="n">high</span>
<span class="k">return</span> <span class="n">obj</span>
<span class="k">def</span> <span class="nf">_create_local_socket</span><span class="p">(</span><span class="n">sock_info</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Create a local socket that can be used to load deserialized data from the JVM</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> sock_info : tuple</span>
<span class="sd"> Tuple containing port number and authentication secret for a local socket.</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> sockfile file descriptor of the local socket</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">port</span> <span class="o">=</span> <span class="n">sock_info</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="n">auth_secret</span> <span class="o">=</span> <span class="n">sock_info</span><span class="p">[</span><span class="mi">1</span><span class="p">]</span>
<span class="n">sockfile</span><span class="p">,</span> <span class="n">sock</span> <span class="o">=</span> <span class="n">local_connect_and_auth</span><span class="p">(</span><span class="n">port</span><span class="p">,</span> <span class="n">auth_secret</span><span class="p">)</span>
<span class="c1"># The RDD materialization time is unpredictable, if we set a timeout for socket reading</span>
<span class="c1"># operation, it will very possibly fail. See SPARK-18281.</span>
<span class="n">sock</span><span class="o">.</span><span class="n">settimeout</span><span class="p">(</span><span class="kc">None</span><span class="p">)</span>
<span class="k">return</span> <span class="n">sockfile</span>
<span class="k">def</span> <span class="nf">_load_from_socket</span><span class="p">(</span><span class="n">sock_info</span><span class="p">,</span> <span class="n">serializer</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Connect to a local socket described by sock_info and use the given serializer to yield data</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> sock_info : tuple</span>
<span class="sd"> Tuple containing port number and authentication secret for a local socket.</span>
<span class="sd"> serializer : :py:class:`Serializer`</span>
<span class="sd"> The PySpark serializer to use</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> result of :py:meth:`Serializer.load_stream`,</span>
<span class="sd"> usually a generator that yields deserialized data</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">sockfile</span> <span class="o">=</span> <span class="n">_create_local_socket</span><span class="p">(</span><span class="n">sock_info</span><span class="p">)</span>
<span class="c1"># The socket will be automatically closed when garbage-collected.</span>
<span class="k">return</span> <span class="n">serializer</span><span class="o">.</span><span class="n">load_stream</span><span class="p">(</span><span class="n">sockfile</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_local_iterator_from_socket</span><span class="p">(</span><span class="n">sock_info</span><span class="p">,</span> <span class="n">serializer</span><span class="p">):</span>
<span class="k">class</span> <span class="nc">PyLocalIterable</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot; Create a synchronous local iterable over a socket &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">_sock_info</span><span class="p">,</span> <span class="n">_serializer</span><span class="p">):</span>
<span class="n">port</span><span class="p">,</span> <span class="n">auth_secret</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">jsocket_auth_server</span> <span class="o">=</span> <span class="n">_sock_info</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_sockfile</span> <span class="o">=</span> <span class="n">_create_local_socket</span><span class="p">((</span><span class="n">port</span><span class="p">,</span> <span class="n">auth_secret</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_serializer</span> <span class="o">=</span> <span class="n">_serializer</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_read_iter</span> <span class="o">=</span> <span class="nb">iter</span><span class="p">([])</span> <span class="c1"># Initialize as empty iterator</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_read_status</span> <span class="o">=</span> <span class="mi">1</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">while</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_status</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="c1"># Request next partition data from Java</span>
<span class="n">write_int</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sockfile</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_sockfile</span><span class="o">.</span><span class="n">flush</span><span class="p">()</span>
<span class="c1"># If response is 1 then there is a partition to read, if 0 then fully consumed</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_read_status</span> <span class="o">=</span> <span class="n">read_int</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_sockfile</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_status</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="c1"># Load the partition data as a stream and read each item</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_read_iter</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_serializer</span><span class="o">.</span><span class="n">load_stream</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_sockfile</span><span class="p">)</span>
<span class="k">for</span> <span class="n">item</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_iter</span><span class="p">:</span>
<span class="k">yield</span> <span class="n">item</span>
<span class="c1"># An error occurred, join serving thread and raise any exceptions from the JVM</span>
<span class="k">elif</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_status</span> <span class="o">==</span> <span class="o">-</span><span class="mi">1</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">jsocket_auth_server</span><span class="o">.</span><span class="n">getResult</span><span class="p">()</span>
<span class="k">def</span> <span class="fm">__del__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># If local iterator is not fully consumed,</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_status</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="c1"># Finish consuming partition data stream</span>
<span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="bp">self</span><span class="o">.</span><span class="n">_read_iter</span><span class="p">:</span>
<span class="k">pass</span>
<span class="c1"># Tell Java to stop sending data and close connection</span>
<span class="n">write_int</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_sockfile</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_sockfile</span><span class="o">.</span><span class="n">flush</span><span class="p">()</span>
<span class="k">except</span> <span class="ne">Exception</span><span class="p">:</span>
<span class="c1"># Ignore any errors, socket is automatically closed when garbage-collected</span>
<span class="k">pass</span>
<span class="k">return</span> <span class="nb">iter</span><span class="p">(</span><span class="n">PyLocalIterable</span><span class="p">(</span><span class="n">sock_info</span><span class="p">,</span> <span class="n">serializer</span><span class="p">))</span>
<span class="k">class</span> <span class="nc">Partitioner</span><span class="p">(</span><span class="nb">object</span><span class="p">):</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">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">numPartitions</span> <span class="o">=</span> <span class="n">numPartitions</span>
<span class="bp">self</span><span class="o">.</span><span class="n">partitionFunc</span> <span class="o">=</span> <span class="n">partitionFunc</span>
<span class="k">def</span> <span class="fm">__eq__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="k">return</span> <span class="p">(</span><span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">Partitioner</span><span class="p">)</span> <span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">numPartitions</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">numPartitions</span>
<span class="ow">and</span> <span class="bp">self</span><span class="o">.</span><span class="n">partitionFunc</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">partitionFunc</span><span class="p">)</span>
<span class="k">def</span> <span class="fm">__call__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">k</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">partitionFunc</span><span class="p">(</span><span class="n">k</span><span class="p">)</span> <span class="o">%</span> <span class="bp">self</span><span class="o">.</span><span class="n">numPartitions</span>
<div class="viewcode-block" id="RDD"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.html#pyspark.RDD">[docs]</a><span class="k">class</span> <span class="nc">RDD</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 Resilient Distributed Dataset (RDD), the basic abstraction in Spark.</span>
<span class="sd"> Represents an immutable, partitioned collection of elements that can be</span>
<span class="sd"> operated on in parallel.</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">jrdd</span><span class="p">,</span> <span class="n">ctx</span><span class="p">,</span> <span class="n">jrdd_deserializer</span><span class="o">=</span><span class="n">AutoBatchedSerializer</span><span class="p">(</span><span class="n">PickleSerializer</span><span class="p">())):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span> <span class="o">=</span> <span class="n">jrdd</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_cached</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_checkpointed</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">has_resource_profile</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span> <span class="o">=</span> <span class="n">ctx</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span> <span class="o">=</span> <span class="n">jrdd_deserializer</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_id</span> <span class="o">=</span> <span class="n">jrdd</span><span class="o">.</span><span class="n">id</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">partitioner</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">def</span> <span class="nf">_pickled</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reserialize</span><span class="p">(</span><span class="n">AutoBatchedSerializer</span><span class="p">(</span><span class="n">PickleSerializer</span><span class="p">()))</span>
<div class="viewcode-block" id="RDD.id"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.id.html#pyspark.RDD.id">[docs]</a> <span class="k">def</span> <span class="nf">id</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> A unique ID for this RDD (within its SparkContext).</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_id</span></div>
<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="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">toString</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">__getnewargs__</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="c1"># This method is called when attempting to pickle an RDD, which is always an error:</span>
<span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span>
<span class="s2">&quot;It appears that you are attempting to broadcast an RDD or reference an RDD from an &quot;</span>
<span class="s2">&quot;action or transformation. RDD transformations and actions can only be invoked by the &quot;</span>
<span class="s2">&quot;driver, not inside of other transformations; for example, &quot;</span>
<span class="s2">&quot;rdd1.map(lambda x: rdd2.values.count() * x) is invalid because the values &quot;</span>
<span class="s2">&quot;transformation and count action cannot be performed inside of the rdd1.map &quot;</span>
<span class="s2">&quot;transformation. For more information, see SPARK-5063.&quot;</span>
<span class="p">)</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">context</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> The :class:`SparkContext` that this RDD was created on.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span>
<div class="viewcode-block" id="RDD.cache"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.cache.html#pyspark.RDD.cache">[docs]</a> <span class="k">def</span> <span class="nf">cache</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Persist this RDD with the default storage level (`MEMORY_ONLY`).</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_cached</span> <span class="o">=</span> <span class="kc">True</span>
<span class="bp">self</span><span class="o">.</span><span class="n">persist</span><span class="p">(</span><span class="n">StorageLevel</span><span class="o">.</span><span class="n">MEMORY_ONLY</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span></div>
<div class="viewcode-block" id="RDD.persist"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.persist.html#pyspark.RDD.persist">[docs]</a> <span class="k">def</span> <span class="nf">persist</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">storageLevel</span><span class="o">=</span><span class="n">StorageLevel</span><span class="o">.</span><span class="n">MEMORY_ONLY</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Set this RDD&#39;s storage level to persist its values across operations</span>
<span class="sd"> after the first time it is computed. This can only be used to assign</span>
<span class="sd"> a new storage level if the RDD does not have a storage level set yet.</span>
<span class="sd"> If no storage level is specified defaults to (`MEMORY_ONLY`).</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([&quot;b&quot;, &quot;a&quot;, &quot;c&quot;])</span>
<span class="sd"> &gt;&gt;&gt; rdd.persist().is_cached</span>
<span class="sd"> True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_cached</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">javaStorageLevel</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_getJavaStorageLevel</span><span class="p">(</span><span class="n">storageLevel</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">persist</span><span class="p">(</span><span class="n">javaStorageLevel</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span></div>
<div class="viewcode-block" id="RDD.unpersist"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.unpersist.html#pyspark.RDD.unpersist">[docs]</a> <span class="k">def</span> <span class="nf">unpersist</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">blocking</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Mark the RDD as non-persistent, and remove all blocks for it from</span>
<span class="sd"> memory and disk.</span>
<span class="sd"> .. versionchanged:: 3.0.0</span>
<span class="sd"> Added optional argument `blocking` to specify whether to block until all</span>
<span class="sd"> blocks are deleted.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_cached</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">unpersist</span><span class="p">(</span><span class="n">blocking</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span></div>
<div class="viewcode-block" id="RDD.checkpoint"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.checkpoint.html#pyspark.RDD.checkpoint">[docs]</a> <span class="k">def</span> <span class="nf">checkpoint</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Mark this RDD for checkpointing. It will be saved to a file inside the</span>
<span class="sd"> checkpoint directory set with :meth:`SparkContext.setCheckpointDir` and</span>
<span class="sd"> all references to its parent RDDs will be removed. This function must</span>
<span class="sd"> be called before any job has been executed on this RDD. It is strongly</span>
<span class="sd"> recommended that this RDD is persisted in memory, otherwise saving it</span>
<span class="sd"> on a file will require recomputation.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_checkpointed</span> <span class="o">=</span> <span class="kc">True</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">()</span><span class="o">.</span><span class="n">checkpoint</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.isCheckpointed"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.isCheckpointed.html#pyspark.RDD.isCheckpointed">[docs]</a> <span class="k">def</span> <span class="nf">isCheckpointed</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return whether this RDD is checkpointed and materialized, either reliably or locally.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">()</span><span class="o">.</span><span class="n">isCheckpointed</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.localCheckpoint"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.localCheckpoint.html#pyspark.RDD.localCheckpoint">[docs]</a> <span class="k">def</span> <span class="nf">localCheckpoint</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Mark this RDD for local checkpointing using Spark&#39;s existing caching layer.</span>
<span class="sd"> This method is for users who wish to truncate RDD lineages while skipping the expensive</span>
<span class="sd"> step of replicating the materialized data in a reliable distributed file system. This is</span>
<span class="sd"> useful for RDDs with long lineages that need to be truncated periodically (e.g. GraphX).</span>
<span class="sd"> Local checkpointing sacrifices fault-tolerance for performance. In particular, checkpointed</span>
<span class="sd"> data is written to ephemeral local storage in the executors instead of to a reliable,</span>
<span class="sd"> fault-tolerant storage. The effect is that if an executor fails during the computation,</span>
<span class="sd"> the checkpointed data may no longer be accessible, causing an irrecoverable job failure.</span>
<span class="sd"> This is NOT safe to use with dynamic allocation, which removes executors along</span>
<span class="sd"> with their cached blocks. If you must use both features, you are advised to set</span>
<span class="sd"> `spark.dynamicAllocation.cachedExecutorIdleTimeout` to a high value.</span>
<span class="sd"> The checkpoint directory set through :meth:`SparkContext.setCheckpointDir` is not used.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">()</span><span class="o">.</span><span class="n">localCheckpoint</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.isLocallyCheckpointed"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.isLocallyCheckpointed.html#pyspark.RDD.isLocallyCheckpointed">[docs]</a> <span class="k">def</span> <span class="nf">isLocallyCheckpointed</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return whether this RDD is marked for local checkpointing.</span>
<span class="sd"> Exposed for testing.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">()</span><span class="o">.</span><span class="n">isLocallyCheckpointed</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.getCheckpointFile"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.getCheckpointFile.html#pyspark.RDD.getCheckpointFile">[docs]</a> <span class="k">def</span> <span class="nf">getCheckpointFile</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Gets the name of the file to which this RDD was checkpointed</span>
<span class="sd"> Not defined if RDD is checkpointed locally.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">checkpointFile</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">()</span><span class="o">.</span><span class="n">getCheckpointFile</span><span class="p">()</span>
<span class="k">if</span> <span class="n">checkpointFile</span><span class="o">.</span><span class="n">isDefined</span><span class="p">():</span>
<span class="k">return</span> <span class="n">checkpointFile</span><span class="o">.</span><span class="n">get</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.map"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.map.html#pyspark.RDD.map">[docs]</a> <span class="k">def</span> <span class="nf">map</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD by applying a function to each element of this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([&quot;b&quot;, &quot;a&quot;, &quot;c&quot;])</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.map(lambda x: (x, 1)).collect())</span>
<span class="sd"> [(&#39;a&#39;, 1), (&#39;b&#39;, 1), (&#39;c&#39;, 1)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">_</span><span class="p">,</span> <span class="n">iterator</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">map</span><span class="p">(</span><span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">f</span><span class="p">),</span> <span class="n">iterator</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.flatMap"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.flatMap.html#pyspark.RDD.flatMap">[docs]</a> <span class="k">def</span> <span class="nf">flatMap</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD by first applying a function to all elements of this</span>
<span class="sd"> RDD, and then flattening the results.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([2, 3, 4])</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.flatMap(lambda x: range(1, x)).collect())</span>
<span class="sd"> [1, 1, 1, 2, 2, 3]</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.flatMap(lambda x: [(x, x), (x, x)]).collect())</span>
<span class="sd"> [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="n">iterator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">chain</span><span class="o">.</span><span class="n">from_iterable</span><span class="p">(</span><span class="nb">map</span><span class="p">(</span><span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">f</span><span class="p">),</span> <span class="n">iterator</span><span class="p">))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.mapPartitions"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.mapPartitions.html#pyspark.RDD.mapPartitions">[docs]</a> <span class="k">def</span> <span class="nf">mapPartitions</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD by applying a function to each partition of this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 2, 3, 4], 2)</span>
<span class="sd"> &gt;&gt;&gt; def f(iterator): yield sum(iterator)</span>
<span class="sd"> &gt;&gt;&gt; rdd.mapPartitions(f).collect()</span>
<span class="sd"> [3, 7]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="n">iterator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">f</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.mapPartitionsWithIndex"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.mapPartitionsWithIndex.html#pyspark.RDD.mapPartitionsWithIndex">[docs]</a> <span class="k">def</span> <span class="nf">mapPartitionsWithIndex</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD by applying a function to each partition of this RDD,</span>
<span class="sd"> while tracking the index of the original partition.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 2, 3, 4], 4)</span>
<span class="sd"> &gt;&gt;&gt; def f(splitIndex, iterator): yield splitIndex</span>
<span class="sd"> &gt;&gt;&gt; rdd.mapPartitionsWithIndex(f).sum()</span>
<span class="sd"> 6</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">PipelinedRDD</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.mapPartitionsWithSplit"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.mapPartitionsWithSplit.html#pyspark.RDD.mapPartitionsWithSplit">[docs]</a> <span class="k">def</span> <span class="nf">mapPartitionsWithSplit</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD by applying a function to each partition of this RDD,</span>
<span class="sd"> while tracking the index of the original partition.</span>
<span class="sd"> .. deprecated:: 0.9.0</span>
<span class="sd"> use :py:meth:`RDD.mapPartitionsWithIndex` instead.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 2, 3, 4], 4)</span>
<span class="sd"> &gt;&gt;&gt; def f(splitIndex, iterator): yield splitIndex</span>
<span class="sd"> &gt;&gt;&gt; rdd.mapPartitionsWithSplit(f).sum()</span>
<span class="sd"> 6</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;mapPartitionsWithSplit is deprecated; use mapPartitionsWithIndex instead&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span><span class="p">,</span> <span class="n">stacklevel</span><span class="o">=</span><span class="mi">2</span>
<span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.getNumPartitions"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.getNumPartitions.html#pyspark.RDD.getNumPartitions">[docs]</a> <span class="k">def</span> <span class="nf">getNumPartitions</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns the number of partitions in RDD</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 2, 3, 4], 2)</span>
<span class="sd"> &gt;&gt;&gt; rdd.getNumPartitions()</span>
<span class="sd"> 2</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">partitions</span><span class="p">()</span><span class="o">.</span><span class="n">size</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.filter"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.filter.html#pyspark.RDD.filter">[docs]</a> <span class="k">def</span> <span class="nf">filter</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD containing only the elements that satisfy a predicate.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 2, 3, 4, 5])</span>
<span class="sd"> &gt;&gt;&gt; rdd.filter(lambda x: x % 2 == 0).collect()</span>
<span class="sd"> [2, 4]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">filter</span><span class="p">(</span><span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">f</span><span class="p">),</span> <span class="n">iterator</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.distinct"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.distinct.html#pyspark.RDD.distinct">[docs]</a> <span class="k">def</span> <span class="nf">distinct</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD containing the distinct elements in this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sorted(sc.parallelize([1, 1, 2, 3]).distinct().collect())</span>
<span class="sd"> [1, 2, 3]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span> \
<span class="o">.</span><span class="n">reduceByKey</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">,</span> <span class="n">_</span><span class="p">:</span> <span class="n">x</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">)</span> \
<span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span></div>
<div class="viewcode-block" id="RDD.sample"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.sample.html#pyspark.RDD.sample">[docs]</a> <span class="k">def</span> <span class="nf">sample</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">withReplacement</span><span class="p">,</span> <span class="n">fraction</span><span class="p">,</span> <span class="n">seed</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a sampled subset of this RDD.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> withReplacement : bool</span>
<span class="sd"> can elements be sampled multiple times (replaced when sampled out)</span>
<span class="sd"> fraction : float</span>
<span class="sd"> expected size of the sample as a fraction of this RDD&#39;s size</span>
<span class="sd"> without replacement: probability that each element is chosen; fraction must be [0, 1]</span>
<span class="sd"> with replacement: expected number of times each element is chosen; fraction must be &gt;= 0</span>
<span class="sd"> seed : int, optional</span>
<span class="sd"> seed for the random number generator</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This is not guaranteed to provide exactly the fraction specified of the total</span>
<span class="sd"> count of the given :class:`DataFrame`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(range(100), 4)</span>
<span class="sd"> &gt;&gt;&gt; 6 &lt;= rdd.sample(False, 0.1, 81).count() &lt;= 14</span>
<span class="sd"> True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">assert</span> <span class="n">fraction</span> <span class="o">&gt;=</span> <span class="mf">0.0</span><span class="p">,</span> <span class="s2">&quot;Negative fraction value: </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="n">fraction</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">RDDSampler</span><span class="p">(</span><span class="n">withReplacement</span><span class="p">,</span> <span class="n">fraction</span><span class="p">,</span> <span class="n">seed</span><span class="p">)</span><span class="o">.</span><span class="n">func</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.randomSplit"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.randomSplit.html#pyspark.RDD.randomSplit">[docs]</a> <span class="k">def</span> <span class="nf">randomSplit</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">weights</span><span class="p">,</span> <span class="n">seed</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Randomly splits this RDD with the provided weights.</span>
<span class="sd"> weights : list</span>
<span class="sd"> weights for splits, will be normalized if they don&#39;t sum to 1</span>
<span class="sd"> seed : int, optional</span>
<span class="sd"> random seed</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> list</span>
<span class="sd"> split RDDs in a list</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(range(500), 1)</span>
<span class="sd"> &gt;&gt;&gt; rdd1, rdd2 = rdd.randomSplit([2, 3], 17)</span>
<span class="sd"> &gt;&gt;&gt; len(rdd1.collect() + rdd2.collect())</span>
<span class="sd"> 500</span>
<span class="sd"> &gt;&gt;&gt; 150 &lt; rdd1.count() &lt; 250</span>
<span class="sd"> True</span>
<span class="sd"> &gt;&gt;&gt; 250 &lt; rdd2.count() &lt; 350</span>
<span class="sd"> True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">s</span> <span class="o">=</span> <span class="nb">float</span><span class="p">(</span><span class="nb">sum</span><span class="p">(</span><span class="n">weights</span><span class="p">))</span>
<span class="n">cweights</span> <span class="o">=</span> <span class="p">[</span><span class="mf">0.0</span><span class="p">]</span>
<span class="k">for</span> <span class="n">w</span> <span class="ow">in</span> <span class="n">weights</span><span class="p">:</span>
<span class="n">cweights</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">cweights</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> <span class="o">+</span> <span class="n">w</span> <span class="o">/</span> <span class="n">s</span><span class="p">)</span>
<span class="k">if</span> <span class="n">seed</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">seed</span> <span class="o">=</span> <span class="n">random</span><span class="o">.</span><span class="n">randint</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="mi">2</span> <span class="o">**</span> <span class="mi">32</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)</span>
<span class="k">return</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">RDDRangeSampler</span><span class="p">(</span><span class="n">lb</span><span class="p">,</span> <span class="n">ub</span><span class="p">,</span> <span class="n">seed</span><span class="p">)</span><span class="o">.</span><span class="n">func</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span>
<span class="k">for</span> <span class="n">lb</span><span class="p">,</span> <span class="n">ub</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">cweights</span><span class="p">,</span> <span class="n">cweights</span><span class="p">[</span><span class="mi">1</span><span class="p">:])]</span></div>
<span class="c1"># this is ported from scala/spark/RDD.scala</span>
<div class="viewcode-block" id="RDD.takeSample"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.takeSample.html#pyspark.RDD.takeSample">[docs]</a> <span class="k">def</span> <span class="nf">takeSample</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">withReplacement</span><span class="p">,</span> <span class="n">num</span><span class="p">,</span> <span class="n">seed</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a fixed-size sampled subset of this RDD.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This method should only be used if the resulting array is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(range(0, 10))</span>
<span class="sd"> &gt;&gt;&gt; len(rdd.takeSample(True, 20, 1))</span>
<span class="sd"> 20</span>
<span class="sd"> &gt;&gt;&gt; len(rdd.takeSample(False, 5, 2))</span>
<span class="sd"> 5</span>
<span class="sd"> &gt;&gt;&gt; len(rdd.takeSample(False, 15, 3))</span>
<span class="sd"> 10</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">numStDev</span> <span class="o">=</span> <span class="mf">10.0</span>
<span class="k">if</span> <span class="n">num</span> <span class="o">&lt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Sample size cannot be negative.&quot;</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">num</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="p">[]</span>
<span class="n">initialCount</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">count</span><span class="p">()</span>
<span class="k">if</span> <span class="n">initialCount</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="p">[]</span>
<span class="n">rand</span> <span class="o">=</span> <span class="n">random</span><span class="o">.</span><span class="n">Random</span><span class="p">(</span><span class="n">seed</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="ow">not</span> <span class="n">withReplacement</span><span class="p">)</span> <span class="ow">and</span> <span class="n">num</span> <span class="o">&gt;=</span> <span class="n">initialCount</span><span class="p">:</span>
<span class="c1"># shuffle current RDD and return</span>
<span class="n">samples</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span>
<span class="n">rand</span><span class="o">.</span><span class="n">shuffle</span><span class="p">(</span><span class="n">samples</span><span class="p">)</span>
<span class="k">return</span> <span class="n">samples</span>
<span class="n">maxSampleSize</span> <span class="o">=</span> <span class="n">sys</span><span class="o">.</span><span class="n">maxsize</span> <span class="o">-</span> <span class="nb">int</span><span class="p">(</span><span class="n">numStDev</span> <span class="o">*</span> <span class="n">sqrt</span><span class="p">(</span><span class="n">sys</span><span class="o">.</span><span class="n">maxsize</span><span class="p">))</span>
<span class="k">if</span> <span class="n">num</span> <span class="o">&gt;</span> <span class="n">maxSampleSize</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span>
<span class="s2">&quot;Sample size cannot be greater than </span><span class="si">%d</span><span class="s2">.&quot;</span> <span class="o">%</span> <span class="n">maxSampleSize</span><span class="p">)</span>
<span class="n">fraction</span> <span class="o">=</span> <span class="n">RDD</span><span class="o">.</span><span class="n">_computeFractionForSampleSize</span><span class="p">(</span>
<span class="n">num</span><span class="p">,</span> <span class="n">initialCount</span><span class="p">,</span> <span class="n">withReplacement</span><span class="p">)</span>
<span class="n">samples</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">sample</span><span class="p">(</span><span class="n">withReplacement</span><span class="p">,</span> <span class="n">fraction</span><span class="p">,</span> <span class="n">seed</span><span class="p">)</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span>
<span class="c1"># If the first sample didn&#39;t turn out large enough, keep trying to take samples;</span>
<span class="c1"># this shouldn&#39;t happen often because we use a big multiplier for their initial size.</span>
<span class="c1"># See: scala/spark/RDD.scala</span>
<span class="k">while</span> <span class="nb">len</span><span class="p">(</span><span class="n">samples</span><span class="p">)</span> <span class="o">&lt;</span> <span class="n">num</span><span class="p">:</span>
<span class="c1"># TODO: add log warning for when more than one iteration was run</span>
<span class="n">seed</span> <span class="o">=</span> <span class="n">rand</span><span class="o">.</span><span class="n">randint</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">sys</span><span class="o">.</span><span class="n">maxsize</span><span class="p">)</span>
<span class="n">samples</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">sample</span><span class="p">(</span><span class="n">withReplacement</span><span class="p">,</span> <span class="n">fraction</span><span class="p">,</span> <span class="n">seed</span><span class="p">)</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span>
<span class="n">rand</span><span class="o">.</span><span class="n">shuffle</span><span class="p">(</span><span class="n">samples</span><span class="p">)</span>
<span class="k">return</span> <span class="n">samples</span><span class="p">[</span><span class="mi">0</span><span class="p">:</span><span class="n">num</span><span class="p">]</span></div>
<span class="nd">@staticmethod</span>
<span class="k">def</span> <span class="nf">_computeFractionForSampleSize</span><span class="p">(</span><span class="n">sampleSizeLowerBound</span><span class="p">,</span> <span class="n">total</span><span class="p">,</span> <span class="n">withReplacement</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns a sampling rate that guarantees a sample of</span>
<span class="sd"> size &gt;= sampleSizeLowerBound 99.99% of the time.</span>
<span class="sd"> How the sampling rate is determined:</span>
<span class="sd"> Let p = num / total, where num is the sample size and total is the</span>
<span class="sd"> total number of data points in the RDD. We&#39;re trying to compute</span>
<span class="sd"> q &gt; p such that</span>
<span class="sd"> - when sampling with replacement, we&#39;re drawing each data point</span>
<span class="sd"> with prob_i ~ Pois(q), where we want to guarantee</span>
<span class="sd"> Pr[s &lt; num] &lt; 0.0001 for s = sum(prob_i for i from 0 to</span>
<span class="sd"> total), i.e. the failure rate of not having a sufficiently large</span>
<span class="sd"> sample &lt; 0.0001. Setting q = p + 5 * sqrt(p/total) is sufficient</span>
<span class="sd"> to guarantee 0.9999 success rate for num &gt; 12, but we need a</span>
<span class="sd"> slightly larger q (9 empirically determined).</span>
<span class="sd"> - when sampling without replacement, we&#39;re drawing each data point</span>
<span class="sd"> with prob_i ~ Binomial(total, fraction) and our choice of q</span>
<span class="sd"> guarantees 1-delta, or 0.9999 success rate, where success rate is</span>
<span class="sd"> defined the same as in sampling with replacement.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">fraction</span> <span class="o">=</span> <span class="nb">float</span><span class="p">(</span><span class="n">sampleSizeLowerBound</span><span class="p">)</span> <span class="o">/</span> <span class="n">total</span>
<span class="k">if</span> <span class="n">withReplacement</span><span class="p">:</span>
<span class="n">numStDev</span> <span class="o">=</span> <span class="mi">5</span>
<span class="k">if</span> <span class="p">(</span><span class="n">sampleSizeLowerBound</span> <span class="o">&lt;</span> <span class="mi">12</span><span class="p">):</span>
<span class="n">numStDev</span> <span class="o">=</span> <span class="mi">9</span>
<span class="k">return</span> <span class="n">fraction</span> <span class="o">+</span> <span class="n">numStDev</span> <span class="o">*</span> <span class="n">sqrt</span><span class="p">(</span><span class="n">fraction</span> <span class="o">/</span> <span class="n">total</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">delta</span> <span class="o">=</span> <span class="mf">0.00005</span>
<span class="n">gamma</span> <span class="o">=</span> <span class="o">-</span> <span class="n">log</span><span class="p">(</span><span class="n">delta</span><span class="p">)</span> <span class="o">/</span> <span class="n">total</span>
<span class="k">return</span> <span class="nb">min</span><span class="p">(</span><span class="mi">1</span><span class="p">,</span> <span class="n">fraction</span> <span class="o">+</span> <span class="n">gamma</span> <span class="o">+</span> <span class="n">sqrt</span><span class="p">(</span><span class="n">gamma</span> <span class="o">*</span> <span class="n">gamma</span> <span class="o">+</span> <span class="mi">2</span> <span class="o">*</span> <span class="n">gamma</span> <span class="o">*</span> <span class="n">fraction</span><span class="p">))</span>
<div class="viewcode-block" id="RDD.union"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.union.html#pyspark.RDD.union">[docs]</a> <span class="k">def</span> <span class="nf">union</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="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the union of this RDD and another one.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 1, 2, 3])</span>
<span class="sd"> &gt;&gt;&gt; rdd.union(rdd).collect()</span>
<span class="sd"> [1, 1, 2, 3, 1, 1, 2, 3]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">:</span>
<span class="n">rdd</span> <span class="o">=</span> <span class="n">RDD</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">union</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">),</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># These RDDs contain data in different serialized formats, so we</span>
<span class="c1"># must normalize them to the default serializer.</span>
<span class="n">self_copy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reserialize</span><span class="p">()</span>
<span class="n">other_copy</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">_reserialize</span><span class="p">()</span>
<span class="n">rdd</span> <span class="o">=</span> <span class="n">RDD</span><span class="p">(</span><span class="n">self_copy</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">union</span><span class="p">(</span><span class="n">other_copy</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">),</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">serializer</span><span class="p">)</span>
<span class="k">if</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">partitioner</span> <span class="o">==</span> <span class="n">other</span><span class="o">.</span><span class="n">partitioner</span> <span class="ow">and</span>
<span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span> <span class="o">==</span> <span class="n">rdd</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()):</span>
<span class="n">rdd</span><span class="o">.</span><span class="n">partitioner</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">partitioner</span>
<span class="k">return</span> <span class="n">rdd</span></div>
<div class="viewcode-block" id="RDD.intersection"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.intersection.html#pyspark.RDD.intersection">[docs]</a> <span class="k">def</span> <span class="nf">intersection</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="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the intersection of this RDD and another one. The output will</span>
<span class="sd"> not contain any duplicate elements, even if the input RDDs did.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This method performs a shuffle internally.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd1 = sc.parallelize([1, 10, 2, 3, 4, 5])</span>
<span class="sd"> &gt;&gt;&gt; rdd2 = sc.parallelize([1, 6, 2, 3, 7, 8])</span>
<span class="sd"> &gt;&gt;&gt; rdd1.intersection(rdd2).collect()</span>
<span class="sd"> [1, 2, 3]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="kc">None</span><span class="p">))</span> \
<span class="o">.</span><span class="n">cogroup</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="p">(</span><span class="n">v</span><span class="p">,</span> <span class="kc">None</span><span class="p">)))</span> \
<span class="o">.</span><span class="n">filter</span><span class="p">(</span><span class="k">lambda</span> <span class="n">k_vs</span><span class="p">:</span> <span class="nb">all</span><span class="p">(</span><span class="n">k_vs</span><span class="p">[</span><span class="mi">1</span><span class="p">]))</span> \
<span class="o">.</span><span class="n">keys</span><span class="p">()</span></div>
<span class="k">def</span> <span class="nf">_reserialize</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">serializer</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="n">serializer</span> <span class="o">=</span> <span class="n">serializer</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">serializer</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span> <span class="o">!=</span> <span class="n">serializer</span><span class="p">:</span>
<span class="bp">self</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span> <span class="o">=</span> <span class="n">serializer</span>
<span class="k">return</span> <span class="bp">self</span>
<span class="k">def</span> <span class="fm">__add__</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the union of this RDD and another one.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 1, 2, 3])</span>
<span class="sd"> &gt;&gt;&gt; (rdd + rdd).collect()</span>
<span class="sd"> [1, 1, 2, 3, 1, 1, 2, 3]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="ow">not</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">RDD</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">TypeError</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">union</span><span class="p">(</span><span class="n">other</span><span class="p">)</span>
<div class="viewcode-block" id="RDD.repartitionAndSortWithinPartitions"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.repartitionAndSortWithinPartitions.html#pyspark.RDD.repartitionAndSortWithinPartitions">[docs]</a> <span class="k">def</span> <span class="nf">repartitionAndSortWithinPartitions</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="o">=</span><span class="n">portable_hash</span><span class="p">,</span>
<span class="n">ascending</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">keyfunc</span><span class="o">=</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Repartition the RDD according to the given partitioner and, within each resulting partition,</span>
<span class="sd"> sort records by their keys.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([(0, 5), (3, 8), (2, 6), (0, 8), (3, 8), (1, 3)])</span>
<span class="sd"> &gt;&gt;&gt; rdd2 = rdd.repartitionAndSortWithinPartitions(2, lambda x: x % 2, True)</span>
<span class="sd"> &gt;&gt;&gt; rdd2.glom().collect()</span>
<span class="sd"> [[(0, 5), (0, 8), (2, 6)], [(1, 3), (3, 8), (3, 8)]]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">numPartitions</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">numPartitions</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_defaultReducePartitions</span><span class="p">()</span>
<span class="n">memory</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_memory_limit</span><span class="p">()</span>
<span class="n">serializer</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span>
<span class="k">def</span> <span class="nf">sortPartition</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">sort</span> <span class="o">=</span> <span class="n">ExternalSorter</span><span class="p">(</span><span class="n">memory</span> <span class="o">*</span> <span class="mf">0.9</span><span class="p">,</span> <span class="n">serializer</span><span class="p">)</span><span class="o">.</span><span class="n">sorted</span>
<span class="k">return</span> <span class="nb">iter</span><span class="p">(</span><span class="n">sort</span><span class="p">(</span><span class="n">iterator</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="k">lambda</span> <span class="n">k_v</span><span class="p">:</span> <span class="n">keyfunc</span><span class="p">(</span><span class="n">k_v</span><span class="p">[</span><span class="mi">0</span><span class="p">]),</span> <span class="n">reverse</span><span class="o">=</span><span class="p">(</span><span class="ow">not</span> <span class="n">ascending</span><span class="p">)))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="p">)</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">sortPartition</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.sortByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.sortByKey.html#pyspark.RDD.sortByKey">[docs]</a> <span class="k">def</span> <span class="nf">sortByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">ascending</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">keyfunc</span><span class="o">=</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Sorts this RDD, which is assumed to consist of (key, value) pairs.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; tmp = [(&#39;a&#39;, 1), (&#39;b&#39;, 2), (&#39;1&#39;, 3), (&#39;d&#39;, 4), (&#39;2&#39;, 5)]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize(tmp).sortByKey().first()</span>
<span class="sd"> (&#39;1&#39;, 3)</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize(tmp).sortByKey(True, 1).collect()</span>
<span class="sd"> [(&#39;1&#39;, 3), (&#39;2&#39;, 5), (&#39;a&#39;, 1), (&#39;b&#39;, 2), (&#39;d&#39;, 4)]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize(tmp).sortByKey(True, 2).collect()</span>
<span class="sd"> [(&#39;1&#39;, 3), (&#39;2&#39;, 5), (&#39;a&#39;, 1), (&#39;b&#39;, 2), (&#39;d&#39;, 4)]</span>
<span class="sd"> &gt;&gt;&gt; tmp2 = [(&#39;Mary&#39;, 1), (&#39;had&#39;, 2), (&#39;a&#39;, 3), (&#39;little&#39;, 4), (&#39;lamb&#39;, 5)]</span>
<span class="sd"> &gt;&gt;&gt; tmp2.extend([(&#39;whose&#39;, 6), (&#39;fleece&#39;, 7), (&#39;was&#39;, 8), (&#39;white&#39;, 9)])</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize(tmp2).sortByKey(True, 3, keyfunc=lambda k: k.lower()).collect()</span>
<span class="sd"> [(&#39;a&#39;, 3), (&#39;fleece&#39;, 7), (&#39;had&#39;, 2), (&#39;lamb&#39;, 5),...(&#39;white&#39;, 9), (&#39;whose&#39;, 6)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">numPartitions</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">numPartitions</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_defaultReducePartitions</span><span class="p">()</span>
<span class="n">memory</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_memory_limit</span><span class="p">()</span>
<span class="n">serializer</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span>
<span class="k">def</span> <span class="nf">sortPartition</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">sort</span> <span class="o">=</span> <span class="n">ExternalSorter</span><span class="p">(</span><span class="n">memory</span> <span class="o">*</span> <span class="mf">0.9</span><span class="p">,</span> <span class="n">serializer</span><span class="p">)</span><span class="o">.</span><span class="n">sorted</span>
<span class="k">return</span> <span class="nb">iter</span><span class="p">(</span><span class="n">sort</span><span class="p">(</span><span class="n">iterator</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="k">lambda</span> <span class="n">kv</span><span class="p">:</span> <span class="n">keyfunc</span><span class="p">(</span><span class="n">kv</span><span class="p">[</span><span class="mi">0</span><span class="p">]),</span> <span class="n">reverse</span><span class="o">=</span><span class="p">(</span><span class="ow">not</span> <span class="n">ascending</span><span class="p">)))</span>
<span class="k">if</span> <span class="n">numPartitions</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="bp">self</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">coalesce</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">sortPartition</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span>
<span class="c1"># first compute the boundary of each part via sampling: we want to partition</span>
<span class="c1"># the key-space into bins such that the bins have roughly the same</span>
<span class="c1"># number of (key, value) pairs falling into them</span>
<span class="n">rddSize</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">count</span><span class="p">()</span>
<span class="k">if</span> <span class="ow">not</span> <span class="n">rddSize</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span> <span class="c1"># empty RDD</span>
<span class="n">maxSampleSize</span> <span class="o">=</span> <span class="n">numPartitions</span> <span class="o">*</span> <span class="mf">20.0</span> <span class="c1"># constant from Spark&#39;s RangePartitioner</span>
<span class="n">fraction</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">maxSampleSize</span> <span class="o">/</span> <span class="nb">max</span><span class="p">(</span><span class="n">rddSize</span><span class="p">,</span> <span class="mi">1</span><span class="p">),</span> <span class="mf">1.0</span><span class="p">)</span>
<span class="n">samples</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">sample</span><span class="p">(</span><span class="kc">False</span><span class="p">,</span> <span class="n">fraction</span><span class="p">,</span> <span class="mi">1</span><span class="p">)</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">kv</span><span class="p">:</span> <span class="n">kv</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span>
<span class="n">samples</span> <span class="o">=</span> <span class="nb">sorted</span><span class="p">(</span><span class="n">samples</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="n">keyfunc</span><span class="p">)</span>
<span class="c1"># we have numPartitions many parts but one of the them has</span>
<span class="c1"># an implicit boundary</span>
<span class="n">bounds</span> <span class="o">=</span> <span class="p">[</span><span class="n">samples</span><span class="p">[</span><span class="nb">int</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">samples</span><span class="p">)</span> <span class="o">*</span> <span class="p">(</span><span class="n">i</span> <span class="o">+</span> <span class="mi">1</span><span class="p">)</span> <span class="o">/</span> <span class="n">numPartitions</span><span class="p">)]</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">numPartitions</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)]</span>
<span class="k">def</span> <span class="nf">rangePartitioner</span><span class="p">(</span><span class="n">k</span><span class="p">):</span>
<span class="n">p</span> <span class="o">=</span> <span class="n">bisect</span><span class="o">.</span><span class="n">bisect_left</span><span class="p">(</span><span class="n">bounds</span><span class="p">,</span> <span class="n">keyfunc</span><span class="p">(</span><span class="n">k</span><span class="p">))</span>
<span class="k">if</span> <span class="n">ascending</span><span class="p">:</span>
<span class="k">return</span> <span class="n">p</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">numPartitions</span> <span class="o">-</span> <span class="mi">1</span> <span class="o">-</span> <span class="n">p</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">rangePartitioner</span><span class="p">)</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">sortPartition</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.sortBy"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.sortBy.html#pyspark.RDD.sortBy">[docs]</a> <span class="k">def</span> <span class="nf">sortBy</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">keyfunc</span><span class="p">,</span> <span class="n">ascending</span><span class="o">=</span><span class="kc">True</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Sorts this RDD by the given keyfunc</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; tmp = [(&#39;a&#39;, 1), (&#39;b&#39;, 2), (&#39;1&#39;, 3), (&#39;d&#39;, 4), (&#39;2&#39;, 5)]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize(tmp).sortBy(lambda x: x[0]).collect()</span>
<span class="sd"> [(&#39;1&#39;, 3), (&#39;2&#39;, 5), (&#39;a&#39;, 1), (&#39;b&#39;, 2), (&#39;d&#39;, 4)]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize(tmp).sortBy(lambda x: x[1]).collect()</span>
<span class="sd"> [(&#39;a&#39;, 1), (&#39;b&#39;, 2), (&#39;1&#39;, 3), (&#39;d&#39;, 4), (&#39;2&#39;, 5)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">keyBy</span><span class="p">(</span><span class="n">keyfunc</span><span class="p">)</span><span class="o">.</span><span class="n">sortByKey</span><span class="p">(</span><span class="n">ascending</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">)</span><span class="o">.</span><span class="n">values</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.glom"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.glom.html#pyspark.RDD.glom">[docs]</a> <span class="k">def</span> <span class="nf">glom</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return an RDD created by coalescing all elements within each partition</span>
<span class="sd"> into a list.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 2, 3, 4], 2)</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.glom().collect())</span>
<span class="sd"> [[1, 2], [3, 4]]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="k">yield</span> <span class="nb">list</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">func</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.cartesian"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.cartesian.html#pyspark.RDD.cartesian">[docs]</a> <span class="k">def</span> <span class="nf">cartesian</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="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the Cartesian product of this RDD and another one, that is, the</span>
<span class="sd"> RDD of all pairs of elements ``(a, b)`` where ``a`` is in `self` and</span>
<span class="sd"> ``b`` is in `other`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 2])</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.cartesian(rdd).collect())</span>
<span class="sd"> [(1, 1), (1, 2), (2, 1), (2, 2)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># Due to batching, we can&#39;t use the Java cartesian method.</span>
<span class="n">deserializer</span> <span class="o">=</span> <span class="n">CartesianDeserializer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">,</span>
<span class="n">other</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">)</span>
<span class="k">return</span> <span class="n">RDD</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">cartesian</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">),</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="p">,</span> <span class="n">deserializer</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.groupBy"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.groupBy.html#pyspark.RDD.groupBy">[docs]</a> <span class="k">def</span> <span class="nf">groupBy</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="o">=</span><span class="n">portable_hash</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return an RDD of grouped items.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 1, 2, 3, 5, 8])</span>
<span class="sd"> &gt;&gt;&gt; result = rdd.groupBy(lambda x: x % 2).collect()</span>
<span class="sd"> &gt;&gt;&gt; sorted([(x, sorted(y)) for (x, y) in result])</span>
<span class="sd"> [(0, [2, 8]), (1, [1, 1, 3, 5])]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">f</span><span class="p">(</span><span class="n">x</span><span class="p">),</span> <span class="n">x</span><span class="p">))</span><span class="o">.</span><span class="n">groupByKey</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.pipe"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.pipe.html#pyspark.RDD.pipe">[docs]</a> <span class="k">def</span> <span class="nf">pipe</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">command</span><span class="p">,</span> <span class="n">env</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">checkCode</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return an RDD created by piping elements to a forked external process.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> command : str</span>
<span class="sd"> command to run.</span>
<span class="sd"> env : dict, optional</span>
<span class="sd"> environment variables to set.</span>
<span class="sd"> checkCode : bool, optional</span>
<span class="sd"> whether or not to check the return value of the shell command.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([&#39;1&#39;, &#39;2&#39;, &#39;&#39;, &#39;3&#39;]).pipe(&#39;cat&#39;).collect()</span>
<span class="sd"> [&#39;1&#39;, &#39;2&#39;, &#39;&#39;, &#39;3&#39;]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">env</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">env</span> <span class="o">=</span> <span class="nb">dict</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">pipe</span> <span class="o">=</span> <span class="n">Popen</span><span class="p">(</span>
<span class="n">shlex</span><span class="o">.</span><span class="n">split</span><span class="p">(</span><span class="n">command</span><span class="p">),</span> <span class="n">env</span><span class="o">=</span><span class="n">env</span><span class="p">,</span> <span class="n">stdin</span><span class="o">=</span><span class="n">PIPE</span><span class="p">,</span> <span class="n">stdout</span><span class="o">=</span><span class="n">PIPE</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">pipe_objs</span><span class="p">(</span><span class="n">out</span><span class="p">):</span>
<span class="k">for</span> <span class="n">obj</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="n">s</span> <span class="o">=</span> <span class="nb">str</span><span class="p">(</span><span class="n">obj</span><span class="p">)</span><span class="o">.</span><span class="n">rstrip</span><span class="p">(</span><span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">)</span> <span class="o">+</span> <span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span>
<span class="n">out</span><span class="o">.</span><span class="n">write</span><span class="p">(</span><span class="n">s</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="s1">&#39;utf-8&#39;</span><span class="p">))</span>
<span class="n">out</span><span class="o">.</span><span class="n">close</span><span class="p">()</span>
<span class="n">Thread</span><span class="p">(</span><span class="n">target</span><span class="o">=</span><span class="n">pipe_objs</span><span class="p">,</span> <span class="n">args</span><span class="o">=</span><span class="p">[</span><span class="n">pipe</span><span class="o">.</span><span class="n">stdin</span><span class="p">])</span><span class="o">.</span><span class="n">start</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">check_return_code</span><span class="p">():</span>
<span class="n">pipe</span><span class="o">.</span><span class="n">wait</span><span class="p">()</span>
<span class="k">if</span> <span class="n">checkCode</span> <span class="ow">and</span> <span class="n">pipe</span><span class="o">.</span><span class="n">returncode</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">RuntimeError</span><span class="p">(</span><span class="s2">&quot;Pipe function `</span><span class="si">%s</span><span class="s2">&#39; exited &quot;</span>
<span class="s2">&quot;with error code </span><span class="si">%d</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="p">(</span><span class="n">command</span><span class="p">,</span> <span class="n">pipe</span><span class="o">.</span><span class="n">returncode</span><span class="p">))</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="mi">0</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">i</span>
<span class="k">return</span> <span class="p">(</span><span class="n">x</span><span class="o">.</span><span class="n">rstrip</span><span class="p">(</span><span class="sa">b</span><span class="s1">&#39;</span><span class="se">\n</span><span class="s1">&#39;</span><span class="p">)</span><span class="o">.</span><span class="n">decode</span><span class="p">(</span><span class="s1">&#39;utf-8&#39;</span><span class="p">)</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span>
<span class="n">chain</span><span class="p">(</span><span class="nb">iter</span><span class="p">(</span><span class="n">pipe</span><span class="o">.</span><span class="n">stdout</span><span class="o">.</span><span class="n">readline</span><span class="p">,</span> <span class="sa">b</span><span class="s1">&#39;&#39;</span><span class="p">),</span> <span class="n">check_return_code</span><span class="p">()))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">func</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.foreach"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.foreach.html#pyspark.RDD.foreach">[docs]</a> <span class="k">def</span> <span class="nf">foreach</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Applies a function to all elements of this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; def f(x): print(x)</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3, 4, 5]).foreach(f)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">f</span> <span class="o">=</span> <span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">f</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">processPartition</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="n">f</span><span class="p">(</span><span class="n">x</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">iter</span><span class="p">([])</span>
<span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">processPartition</span><span class="p">)</span><span class="o">.</span><span class="n">count</span><span class="p">()</span> <span class="c1"># Force evaluation</span></div>
<div class="viewcode-block" id="RDD.foreachPartition"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.foreachPartition.html#pyspark.RDD.foreachPartition">[docs]</a> <span class="k">def</span> <span class="nf">foreachPartition</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Applies a function to each partition of this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; def f(iterator):</span>
<span class="sd"> ... for x in iterator:</span>
<span class="sd"> ... print(x)</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3, 4, 5]).foreachPartition(f)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">it</span><span class="p">):</span>
<span class="n">r</span> <span class="o">=</span> <span class="n">f</span><span class="p">(</span><span class="n">it</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">iter</span><span class="p">(</span><span class="n">r</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">TypeError</span><span class="p">:</span>
<span class="k">return</span> <span class="nb">iter</span><span class="p">([])</span>
<span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">func</span><span class="p">)</span><span class="o">.</span><span class="n">count</span><span class="p">()</span> <span class="c1"># Force evaluation</span></div>
<div class="viewcode-block" id="RDD.collect"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.collect.html#pyspark.RDD.collect">[docs]</a> <span class="k">def</span> <span class="nf">collect</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a list that contains all of the elements in this RDD.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This method should only be used if the resulting array is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">with</span> <span class="n">SCCallSiteSync</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">context</span><span class="p">)</span> <span class="k">as</span> <span class="n">css</span><span class="p">:</span>
<span class="n">sock_info</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">collectAndServe</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">())</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="n">_load_from_socket</span><span class="p">(</span><span class="n">sock_info</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">))</span></div>
<div class="viewcode-block" id="RDD.collectWithJobGroup"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.collectWithJobGroup.html#pyspark.RDD.collectWithJobGroup">[docs]</a> <span class="k">def</span> <span class="nf">collectWithJobGroup</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">groupId</span><span class="p">,</span> <span class="n">description</span><span class="p">,</span> <span class="n">interruptOnCancel</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> When collect rdd, use this method to specify job group.</span>
<span class="sd"> .. versionadded:: 3.0.0</span>
<span class="sd"> .. deprecated:: 3.1.0</span>
<span class="sd"> Use :class:`pyspark.InheritableThread` with the pinned thread mode enabled.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">warnings</span><span class="o">.</span><span class="n">warn</span><span class="p">(</span>
<span class="s2">&quot;Deprecated in 3.1, Use pyspark.InheritableThread with &quot;</span>
<span class="s2">&quot;the pinned thread mode enabled.&quot;</span><span class="p">,</span>
<span class="ne">FutureWarning</span>
<span class="p">)</span>
<span class="k">with</span> <span class="n">SCCallSiteSync</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">context</span><span class="p">)</span> <span class="k">as</span> <span class="n">css</span><span class="p">:</span>
<span class="n">sock_info</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">collectAndServeWithJobGroup</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">(),</span> <span class="n">groupId</span><span class="p">,</span> <span class="n">description</span><span class="p">,</span> <span class="n">interruptOnCancel</span><span class="p">)</span>
<span class="k">return</span> <span class="nb">list</span><span class="p">(</span><span class="n">_load_from_socket</span><span class="p">(</span><span class="n">sock_info</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">))</span></div>
<div class="viewcode-block" id="RDD.reduce"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.reduce.html#pyspark.RDD.reduce">[docs]</a> <span class="k">def</span> <span class="nf">reduce</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Reduces the elements of this RDD using the specified commutative and</span>
<span class="sd"> associative binary operator. Currently reduces partitions locally.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from operator import add</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3, 4, 5]).reduce(add)</span>
<span class="sd"> 15</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize((2 for _ in range(10))).map(lambda x: 1).cache().reduce(add)</span>
<span class="sd"> 10</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([]).reduce(add)</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: Can not reduce() empty RDD</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">f</span> <span class="o">=</span> <span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">f</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">iterator</span> <span class="o">=</span> <span class="nb">iter</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">initial</span> <span class="o">=</span> <span class="nb">next</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">StopIteration</span><span class="p">:</span>
<span class="k">return</span>
<span class="k">yield</span> <span class="n">reduce</span><span class="p">(</span><span class="n">f</span><span class="p">,</span> <span class="n">iterator</span><span class="p">,</span> <span class="n">initial</span><span class="p">)</span>
<span class="n">vals</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">func</span><span class="p">)</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span>
<span class="k">if</span> <span class="n">vals</span><span class="p">:</span>
<span class="k">return</span> <span class="n">reduce</span><span class="p">(</span><span class="n">f</span><span class="p">,</span> <span class="n">vals</span><span class="p">)</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Can not reduce() empty RDD&quot;</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.treeReduce"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.treeReduce.html#pyspark.RDD.treeReduce">[docs]</a> <span class="k">def</span> <span class="nf">treeReduce</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">depth</span><span class="o">=</span><span class="mi">2</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Reduces the elements of this RDD in a multi-level tree pattern.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> f : function</span>
<span class="sd"> depth : int, optional</span>
<span class="sd"> suggested depth of the tree (default: 2)</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; add = lambda x, y: x + y</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([-5, -4, -3, -2, -1, 1, 2, 3, 4], 10)</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeReduce(add)</span>
<span class="sd"> -5</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeReduce(add, 1)</span>
<span class="sd"> -5</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeReduce(add, 2)</span>
<span class="sd"> -5</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeReduce(add, 5)</span>
<span class="sd"> -5</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeReduce(add, 10)</span>
<span class="sd"> -5</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">depth</span> <span class="o">&lt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Depth cannot be smaller than 1 but got </span><span class="si">%d</span><span class="s2">.&quot;</span> <span class="o">%</span> <span class="n">depth</span><span class="p">)</span>
<span class="n">zeroValue</span> <span class="o">=</span> <span class="kc">None</span><span class="p">,</span> <span class="kc">True</span> <span class="c1"># Use the second entry to indicate whether this is a dummy value.</span>
<span class="k">def</span> <span class="nf">op</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">y</span><span class="p">):</span>
<span class="k">if</span> <span class="n">x</span><span class="p">[</span><span class="mi">1</span><span class="p">]:</span>
<span class="k">return</span> <span class="n">y</span>
<span class="k">elif</span> <span class="n">y</span><span class="p">[</span><span class="mi">1</span><span class="p">]:</span>
<span class="k">return</span> <span class="n">x</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="n">f</span><span class="p">(</span><span class="n">x</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">y</span><span class="p">[</span><span class="mi">0</span><span class="p">]),</span> <span class="kc">False</span>
<span class="n">reduced</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="kc">False</span><span class="p">))</span><span class="o">.</span><span class="n">treeAggregate</span><span class="p">(</span><span class="n">zeroValue</span><span class="p">,</span> <span class="n">op</span><span class="p">,</span> <span class="n">op</span><span class="p">,</span> <span class="n">depth</span><span class="p">)</span>
<span class="k">if</span> <span class="n">reduced</span><span class="p">[</span><span class="mi">1</span><span class="p">]:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Cannot reduce empty RDD.&quot;</span><span class="p">)</span>
<span class="k">return</span> <span class="n">reduced</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span></div>
<div class="viewcode-block" id="RDD.fold"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.fold.html#pyspark.RDD.fold">[docs]</a> <span class="k">def</span> <span class="nf">fold</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">zeroValue</span><span class="p">,</span> <span class="n">op</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Aggregate the elements of each partition, and then the results for all</span>
<span class="sd"> the partitions, using a given associative function and a neutral &quot;zero value.&quot;</span>
<span class="sd"> The function ``op(t1, t2)`` is allowed to modify ``t1`` and return it</span>
<span class="sd"> as its result value to avoid object allocation; however, it should not</span>
<span class="sd"> modify ``t2``.</span>
<span class="sd"> This behaves somewhat differently from fold operations implemented</span>
<span class="sd"> for non-distributed collections in functional languages like Scala.</span>
<span class="sd"> This fold operation may be applied to partitions individually, and then</span>
<span class="sd"> fold those results into the final result, rather than apply the fold</span>
<span class="sd"> to each element sequentially in some defined ordering. For functions</span>
<span class="sd"> that are not commutative, the result may differ from that of a fold</span>
<span class="sd"> applied to a non-distributed collection.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from operator import add</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3, 4, 5]).fold(0, add)</span>
<span class="sd"> 15</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">op</span> <span class="o">=</span> <span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">op</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">acc</span> <span class="o">=</span> <span class="n">zeroValue</span>
<span class="k">for</span> <span class="n">obj</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="n">acc</span> <span class="o">=</span> <span class="n">op</span><span class="p">(</span><span class="n">acc</span><span class="p">,</span> <span class="n">obj</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">acc</span>
<span class="c1"># collecting result of mapPartitions here ensures that the copy of</span>
<span class="c1"># zeroValue provided to each partition is unique from the one provided</span>
<span class="c1"># to the final reduce call</span>
<span class="n">vals</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">func</span><span class="p">)</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span>
<span class="k">return</span> <span class="n">reduce</span><span class="p">(</span><span class="n">op</span><span class="p">,</span> <span class="n">vals</span><span class="p">,</span> <span class="n">zeroValue</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.aggregate"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.aggregate.html#pyspark.RDD.aggregate">[docs]</a> <span class="k">def</span> <span class="nf">aggregate</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">zeroValue</span><span class="p">,</span> <span class="n">seqOp</span><span class="p">,</span> <span class="n">combOp</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Aggregate the elements of each partition, and then the results for all</span>
<span class="sd"> the partitions, using a given combine functions and a neutral &quot;zero</span>
<span class="sd"> value.&quot;</span>
<span class="sd"> The functions ``op(t1, t2)`` is allowed to modify ``t1`` and return it</span>
<span class="sd"> as its result value to avoid object allocation; however, it should not</span>
<span class="sd"> modify ``t2``.</span>
<span class="sd"> The first function (seqOp) can return a different result type, U, than</span>
<span class="sd"> the type of this RDD. Thus, we need one operation for merging a T into</span>
<span class="sd"> an U and one operation for merging two U</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; seqOp = (lambda x, y: (x[0] + y, x[1] + 1))</span>
<span class="sd"> &gt;&gt;&gt; combOp = (lambda x, y: (x[0] + y[0], x[1] + y[1]))</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3, 4]).aggregate((0, 0), seqOp, combOp)</span>
<span class="sd"> (10, 4)</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([]).aggregate((0, 0), seqOp, combOp)</span>
<span class="sd"> (0, 0)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">seqOp</span> <span class="o">=</span> <span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">seqOp</span><span class="p">)</span>
<span class="n">combOp</span> <span class="o">=</span> <span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">combOp</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">acc</span> <span class="o">=</span> <span class="n">zeroValue</span>
<span class="k">for</span> <span class="n">obj</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="n">acc</span> <span class="o">=</span> <span class="n">seqOp</span><span class="p">(</span><span class="n">acc</span><span class="p">,</span> <span class="n">obj</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">acc</span>
<span class="c1"># collecting result of mapPartitions here ensures that the copy of</span>
<span class="c1"># zeroValue provided to each partition is unique from the one provided</span>
<span class="c1"># to the final reduce call</span>
<span class="n">vals</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">func</span><span class="p">)</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span>
<span class="k">return</span> <span class="n">reduce</span><span class="p">(</span><span class="n">combOp</span><span class="p">,</span> <span class="n">vals</span><span class="p">,</span> <span class="n">zeroValue</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.treeAggregate"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.treeAggregate.html#pyspark.RDD.treeAggregate">[docs]</a> <span class="k">def</span> <span class="nf">treeAggregate</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">zeroValue</span><span class="p">,</span> <span class="n">seqOp</span><span class="p">,</span> <span class="n">combOp</span><span class="p">,</span> <span class="n">depth</span><span class="o">=</span><span class="mi">2</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Aggregates the elements of this RDD in a multi-level tree</span>
<span class="sd"> pattern.</span>
<span class="sd"> depth : int, optional</span>
<span class="sd"> suggested depth of the tree (default: 2)</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; add = lambda x, y: x + y</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([-5, -4, -3, -2, -1, 1, 2, 3, 4], 10)</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeAggregate(0, add, add)</span>
<span class="sd"> -5</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeAggregate(0, add, add, 1)</span>
<span class="sd"> -5</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeAggregate(0, add, add, 2)</span>
<span class="sd"> -5</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeAggregate(0, add, add, 5)</span>
<span class="sd"> -5</span>
<span class="sd"> &gt;&gt;&gt; rdd.treeAggregate(0, add, add, 10)</span>
<span class="sd"> -5</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">depth</span> <span class="o">&lt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Depth cannot be smaller than 1 but got </span><span class="si">%d</span><span class="s2">.&quot;</span> <span class="o">%</span> <span class="n">depth</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="k">return</span> <span class="n">zeroValue</span>
<span class="k">def</span> <span class="nf">aggregatePartition</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">acc</span> <span class="o">=</span> <span class="n">zeroValue</span>
<span class="k">for</span> <span class="n">obj</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="n">acc</span> <span class="o">=</span> <span class="n">seqOp</span><span class="p">(</span><span class="n">acc</span><span class="p">,</span> <span class="n">obj</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">acc</span>
<span class="n">partiallyAggregated</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">aggregatePartition</span><span class="p">)</span>
<span class="n">numPartitions</span> <span class="o">=</span> <span class="n">partiallyAggregated</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span>
<span class="n">scale</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="nb">int</span><span class="p">(</span><span class="n">ceil</span><span class="p">(</span><span class="nb">pow</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="mf">1.0</span> <span class="o">/</span> <span class="n">depth</span><span class="p">))),</span> <span class="mi">2</span><span class="p">)</span>
<span class="c1"># If creating an extra level doesn&#39;t help reduce the wall-clock time, we stop the tree</span>
<span class="c1"># aggregation.</span>
<span class="k">while</span> <span class="n">numPartitions</span> <span class="o">&gt;</span> <span class="n">scale</span> <span class="o">+</span> <span class="n">numPartitions</span> <span class="o">/</span> <span class="n">scale</span><span class="p">:</span>
<span class="n">numPartitions</span> <span class="o">/=</span> <span class="n">scale</span>
<span class="n">curNumPartitions</span> <span class="o">=</span> <span class="nb">int</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">mapPartition</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="n">iterator</span><span class="p">):</span>
<span class="k">for</span> <span class="n">obj</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="k">yield</span> <span class="p">(</span><span class="n">i</span> <span class="o">%</span> <span class="n">curNumPartitions</span><span class="p">,</span> <span class="n">obj</span><span class="p">)</span>
<span class="n">partiallyAggregated</span> <span class="o">=</span> <span class="n">partiallyAggregated</span> \
<span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">mapPartition</span><span class="p">)</span> \
<span class="o">.</span><span class="n">reduceByKey</span><span class="p">(</span><span class="n">combOp</span><span class="p">,</span> <span class="n">curNumPartitions</span><span class="p">)</span> \
<span class="o">.</span><span class="n">values</span><span class="p">()</span>
<span class="k">return</span> <span class="n">partiallyAggregated</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="n">combOp</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.max"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.max.html#pyspark.RDD.max">[docs]</a> <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="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Find the maximum item in this RDD.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> key : function, optional</span>
<span class="sd"> A function used to generate key for comparing</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1.0, 5.0, 43.0, 10.0])</span>
<span class="sd"> &gt;&gt;&gt; rdd.max()</span>
<span class="sd"> 43.0</span>
<span class="sd"> &gt;&gt;&gt; rdd.max(key=str)</span>
<span class="sd"> 5.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">key</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="nb">max</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="k">lambda</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">:</span> <span class="nb">max</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="n">key</span><span class="p">))</span></div>
<div class="viewcode-block" id="RDD.min"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.min.html#pyspark.RDD.min">[docs]</a> <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="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Find the minimum item in this RDD.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> key : function, optional</span>
<span class="sd"> A function used to generate key for comparing</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([2.0, 5.0, 43.0, 10.0])</span>
<span class="sd"> &gt;&gt;&gt; rdd.min()</span>
<span class="sd"> 2.0</span>
<span class="sd"> &gt;&gt;&gt; rdd.min(key=str)</span>
<span class="sd"> 10.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">key</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="nb">min</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="k">lambda</span> <span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">:</span> <span class="nb">min</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="n">key</span><span class="p">))</span></div>
<div class="viewcode-block" id="RDD.sum"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.sum.html#pyspark.RDD.sum">[docs]</a> <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="sd">&quot;&quot;&quot;</span>
<span class="sd"> Add up the elements in this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1.0, 2.0, 3.0]).sum()</span>
<span class="sd"> 6.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">[</span><span class="nb">sum</span><span class="p">(</span><span class="n">x</span><span class="p">)])</span><span class="o">.</span><span class="n">fold</span><span class="p">(</span><span class="mi">0</span><span class="p">,</span> <span class="n">operator</span><span class="o">.</span><span class="n">add</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.count"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.count.html#pyspark.RDD.count">[docs]</a> <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="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the number of elements in this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([2, 3, 4]).count()</span>
<span class="sd"> 3</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="k">lambda</span> <span class="n">i</span><span class="p">:</span> <span class="p">[</span><span class="nb">sum</span><span class="p">(</span><span class="mi">1</span> <span class="k">for</span> <span class="n">_</span> <span class="ow">in</span> <span class="n">i</span><span class="p">)])</span><span class="o">.</span><span class="n">sum</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.stats"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.stats.html#pyspark.RDD.stats">[docs]</a> <span class="k">def</span> <span class="nf">stats</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a :class:`StatCounter` object that captures the mean, variance</span>
<span class="sd"> and count of the RDD&#39;s elements in one operation.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">redFunc</span><span class="p">(</span><span class="n">left_counter</span><span class="p">,</span> <span class="n">right_counter</span><span class="p">):</span>
<span class="k">return</span> <span class="n">left_counter</span><span class="o">.</span><span class="n">mergeStats</span><span class="p">(</span><span class="n">right_counter</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="k">lambda</span> <span class="n">i</span><span class="p">:</span> <span class="p">[</span><span class="n">StatCounter</span><span class="p">(</span><span class="n">i</span><span class="p">)])</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="n">redFunc</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.histogram"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.histogram.html#pyspark.RDD.histogram">[docs]</a> <span class="k">def</span> <span class="nf">histogram</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">buckets</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute a histogram using the provided buckets. The buckets</span>
<span class="sd"> are all open to the right except for the last which is closed.</span>
<span class="sd"> e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50],</span>
<span class="sd"> which means 1&lt;=x&lt;10, 10&lt;=x&lt;20, 20&lt;=x&lt;=50. And on the input of 1</span>
<span class="sd"> and 50 we would have a histogram of 1,0,1.</span>
<span class="sd"> If your histogram is evenly spaced (e.g. [0, 10, 20, 30]),</span>
<span class="sd"> this can be switched from an O(log n) insertion to O(1) per</span>
<span class="sd"> element (where n is the number of buckets).</span>
<span class="sd"> Buckets must be sorted, not contain any duplicates, and have</span>
<span class="sd"> at least two elements.</span>
<span class="sd"> If `buckets` is a number, it will generate buckets which are</span>
<span class="sd"> evenly spaced between the minimum and maximum of the RDD. For</span>
<span class="sd"> example, if the min value is 0 and the max is 100, given `buckets`</span>
<span class="sd"> as 2, the resulting buckets will be [0,50) [50,100]. `buckets` must</span>
<span class="sd"> be at least 1. An exception is raised if the RDD contains infinity.</span>
<span class="sd"> If the elements in the RDD do not vary (max == min), a single bucket</span>
<span class="sd"> will be used.</span>
<span class="sd"> The return value is a tuple of buckets and histogram.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(range(51))</span>
<span class="sd"> &gt;&gt;&gt; rdd.histogram(2)</span>
<span class="sd"> ([0, 25, 50], [25, 26])</span>
<span class="sd"> &gt;&gt;&gt; rdd.histogram([0, 5, 25, 50])</span>
<span class="sd"> ([0, 5, 25, 50], [5, 20, 26])</span>
<span class="sd"> &gt;&gt;&gt; rdd.histogram([0, 15, 30, 45, 60]) # evenly spaced buckets</span>
<span class="sd"> ([0, 15, 30, 45, 60], [15, 15, 15, 6])</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([&quot;ab&quot;, &quot;ac&quot;, &quot;b&quot;, &quot;bd&quot;, &quot;ef&quot;])</span>
<span class="sd"> &gt;&gt;&gt; rdd.histogram((&quot;a&quot;, &quot;b&quot;, &quot;c&quot;))</span>
<span class="sd"> ((&#39;a&#39;, &#39;b&#39;, &#39;c&#39;), [2, 2])</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">buckets</span><span class="p">,</span> <span class="nb">int</span><span class="p">):</span>
<span class="k">if</span> <span class="n">buckets</span> <span class="o">&lt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;number of buckets must be &gt;= 1&quot;</span><span class="p">)</span>
<span class="c1"># filter out non-comparable elements</span>
<span class="k">def</span> <span class="nf">comparable</span><span class="p">(</span><span class="n">x</span><span class="p">):</span>
<span class="k">if</span> <span class="n">x</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">if</span> <span class="nb">type</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> <span class="ow">is</span> <span class="nb">float</span> <span class="ow">and</span> <span class="n">isnan</span><span class="p">(</span><span class="n">x</span><span class="p">):</span>
<span class="k">return</span> <span class="kc">False</span>
<span class="k">return</span> <span class="kc">True</span>
<span class="n">filtered</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">filter</span><span class="p">(</span><span class="n">comparable</span><span class="p">)</span>
<span class="c1"># faster than stats()</span>
<span class="k">def</span> <span class="nf">minmax</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">):</span>
<span class="k">return</span> <span class="nb">min</span><span class="p">(</span><span class="n">a</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">b</span><span class="p">[</span><span class="mi">0</span><span class="p">]),</span> <span class="nb">max</span><span class="p">(</span><span class="n">a</span><span class="p">[</span><span class="mi">1</span><span class="p">],</span> <span class="n">b</span><span class="p">[</span><span class="mi">1</span><span class="p">])</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">minv</span><span class="p">,</span> <span class="n">maxv</span> <span class="o">=</span> <span class="n">filtered</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="n">x</span><span class="p">))</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="n">minmax</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">TypeError</span> <span class="k">as</span> <span class="n">e</span><span class="p">:</span>
<span class="k">if</span> <span class="s2">&quot; empty &quot;</span> <span class="ow">in</span> <span class="nb">str</span><span class="p">(</span><span class="n">e</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;can not generate buckets from empty RDD&quot;</span><span class="p">)</span>
<span class="k">raise</span>
<span class="k">if</span> <span class="n">minv</span> <span class="o">==</span> <span class="n">maxv</span> <span class="ow">or</span> <span class="n">buckets</span> <span class="o">==</span> <span class="mi">1</span><span class="p">:</span>
<span class="k">return</span> <span class="p">[</span><span class="n">minv</span><span class="p">,</span> <span class="n">maxv</span><span class="p">],</span> <span class="p">[</span><span class="n">filtered</span><span class="o">.</span><span class="n">count</span><span class="p">()]</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">inc</span> <span class="o">=</span> <span class="p">(</span><span class="n">maxv</span> <span class="o">-</span> <span class="n">minv</span><span class="p">)</span> <span class="o">/</span> <span class="n">buckets</span>
<span class="k">except</span> <span class="ne">TypeError</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;Can not generate buckets with non-number in RDD&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="n">isinf</span><span class="p">(</span><span class="n">inc</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Can not generate buckets with infinite value&quot;</span><span class="p">)</span>
<span class="c1"># keep them as integer if possible</span>
<span class="n">inc</span> <span class="o">=</span> <span class="nb">int</span><span class="p">(</span><span class="n">inc</span><span class="p">)</span>
<span class="k">if</span> <span class="n">inc</span> <span class="o">*</span> <span class="n">buckets</span> <span class="o">!=</span> <span class="n">maxv</span> <span class="o">-</span> <span class="n">minv</span><span class="p">:</span>
<span class="n">inc</span> <span class="o">=</span> <span class="p">(</span><span class="n">maxv</span> <span class="o">-</span> <span class="n">minv</span><span class="p">)</span> <span class="o">*</span> <span class="mf">1.0</span> <span class="o">/</span> <span class="n">buckets</span>
<span class="n">buckets</span> <span class="o">=</span> <span class="p">[</span><span class="n">i</span> <span class="o">*</span> <span class="n">inc</span> <span class="o">+</span> <span class="n">minv</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="n">buckets</span><span class="p">)]</span>
<span class="n">buckets</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">maxv</span><span class="p">)</span> <span class="c1"># fix accumulated error</span>
<span class="n">even</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">elif</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">buckets</span><span class="p">,</span> <span class="p">(</span><span class="nb">list</span><span class="p">,</span> <span class="nb">tuple</span><span class="p">)):</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">buckets</span><span class="p">)</span> <span class="o">&lt;</span> <span class="mi">2</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;buckets should have more than one value&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">any</span><span class="p">(</span><span class="n">i</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">i</span><span class="p">,</span> <span class="nb">float</span><span class="p">)</span> <span class="ow">and</span> <span class="n">isnan</span><span class="p">(</span><span class="n">i</span><span class="p">)</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">buckets</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;can not have None or NaN in buckets&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">sorted</span><span class="p">(</span><span class="n">buckets</span><span class="p">)</span> <span class="o">!=</span> <span class="nb">list</span><span class="p">(</span><span class="n">buckets</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;buckets should be sorted&quot;</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="nb">set</span><span class="p">(</span><span class="n">buckets</span><span class="p">))</span> <span class="o">!=</span> <span class="nb">len</span><span class="p">(</span><span class="n">buckets</span><span class="p">):</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;buckets should not contain duplicated values&quot;</span><span class="p">)</span>
<span class="n">minv</span> <span class="o">=</span> <span class="n">buckets</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="n">maxv</span> <span class="o">=</span> <span class="n">buckets</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span>
<span class="n">even</span> <span class="o">=</span> <span class="kc">False</span>
<span class="n">inc</span> <span class="o">=</span> <span class="kc">None</span>
<span class="k">try</span><span class="p">:</span>
<span class="n">steps</span> <span class="o">=</span> <span class="p">[</span><span class="n">buckets</span><span class="p">[</span><span class="n">i</span> <span class="o">+</span> <span class="mi">1</span><span class="p">]</span> <span class="o">-</span> <span class="n">buckets</span><span class="p">[</span><span class="n">i</span><span class="p">]</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">buckets</span><span class="p">)</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)]</span>
<span class="k">except</span> <span class="ne">TypeError</span><span class="p">:</span>
<span class="k">pass</span> <span class="c1"># objects in buckets do not support &#39;-&#39;</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">if</span> <span class="nb">max</span><span class="p">(</span><span class="n">steps</span><span class="p">)</span> <span class="o">-</span> <span class="nb">min</span><span class="p">(</span><span class="n">steps</span><span class="p">)</span> <span class="o">&lt;</span> <span class="mf">1e-10</span><span class="p">:</span> <span class="c1"># handle precision errors</span>
<span class="n">even</span> <span class="o">=</span> <span class="kc">True</span>
<span class="n">inc</span> <span class="o">=</span> <span class="p">(</span><span class="n">maxv</span> <span class="o">-</span> <span class="n">minv</span><span class="p">)</span> <span class="o">/</span> <span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">buckets</span><span class="p">)</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">TypeError</span><span class="p">(</span><span class="s2">&quot;buckets should be a list or tuple or number(int or long)&quot;</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">histogram</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">counters</span> <span class="o">=</span> <span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">*</span> <span class="nb">len</span><span class="p">(</span><span class="n">buckets</span><span class="p">)</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="k">if</span> <span class="n">i</span> <span class="ow">is</span> <span class="kc">None</span> <span class="ow">or</span> <span class="p">(</span><span class="nb">type</span><span class="p">(</span><span class="n">i</span><span class="p">)</span> <span class="ow">is</span> <span class="nb">float</span> <span class="ow">and</span> <span class="n">isnan</span><span class="p">(</span><span class="n">i</span><span class="p">))</span> <span class="ow">or</span> <span class="n">i</span> <span class="o">&gt;</span> <span class="n">maxv</span> <span class="ow">or</span> <span class="n">i</span> <span class="o">&lt;</span> <span class="n">minv</span><span class="p">:</span>
<span class="k">continue</span>
<span class="n">t</span> <span class="o">=</span> <span class="p">(</span><span class="nb">int</span><span class="p">((</span><span class="n">i</span> <span class="o">-</span> <span class="n">minv</span><span class="p">)</span> <span class="o">/</span> <span class="n">inc</span><span class="p">)</span> <span class="k">if</span> <span class="n">even</span>
<span class="k">else</span> <span class="n">bisect</span><span class="o">.</span><span class="n">bisect_right</span><span class="p">(</span><span class="n">buckets</span><span class="p">,</span> <span class="n">i</span><span class="p">)</span> <span class="o">-</span> <span class="mi">1</span><span class="p">)</span>
<span class="n">counters</span><span class="p">[</span><span class="n">t</span><span class="p">]</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="c1"># add last two together</span>
<span class="n">last</span> <span class="o">=</span> <span class="n">counters</span><span class="o">.</span><span class="n">pop</span><span class="p">()</span>
<span class="n">counters</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> <span class="o">+=</span> <span class="n">last</span>
<span class="k">return</span> <span class="p">[</span><span class="n">counters</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">mergeCounters</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span><span class="n">i</span> <span class="o">+</span> <span class="n">j</span> <span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">j</span> <span class="ow">in</span> <span class="nb">zip</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">)]</span>
<span class="k">return</span> <span class="n">buckets</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">histogram</span><span class="p">)</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="n">mergeCounters</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.mean"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.mean.html#pyspark.RDD.mean">[docs]</a> <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="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute the mean of this RDD&#39;s elements.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3]).mean()</span>
<span class="sd"> 2.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">stats</span><span class="p">()</span><span class="o">.</span><span class="n">mean</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.variance"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.variance.html#pyspark.RDD.variance">[docs]</a> <span class="k">def</span> <span class="nf">variance</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute the variance of this RDD&#39;s elements.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3]).variance()</span>
<span class="sd"> 0.666...</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">stats</span><span class="p">()</span><span class="o">.</span><span class="n">variance</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.stdev"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.stdev.html#pyspark.RDD.stdev">[docs]</a> <span class="k">def</span> <span class="nf">stdev</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute the standard deviation of this RDD&#39;s elements.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3]).stdev()</span>
<span class="sd"> 0.816...</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">stats</span><span class="p">()</span><span class="o">.</span><span class="n">stdev</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.sampleStdev"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.sampleStdev.html#pyspark.RDD.sampleStdev">[docs]</a> <span class="k">def</span> <span class="nf">sampleStdev</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute the sample standard deviation of this RDD&#39;s elements (which</span>
<span class="sd"> corrects for bias in estimating the standard deviation by dividing by</span>
<span class="sd"> N-1 instead of N).</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3]).sampleStdev()</span>
<span class="sd"> 1.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">stats</span><span class="p">()</span><span class="o">.</span><span class="n">sampleStdev</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.sampleVariance"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.sampleVariance.html#pyspark.RDD.sampleVariance">[docs]</a> <span class="k">def</span> <span class="nf">sampleVariance</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Compute the sample variance of this RDD&#39;s elements (which corrects</span>
<span class="sd"> for bias in estimating the variance by dividing by N-1 instead of N).</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3]).sampleVariance()</span>
<span class="sd"> 1.0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">stats</span><span class="p">()</span><span class="o">.</span><span class="n">sampleVariance</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.countByValue"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.countByValue.html#pyspark.RDD.countByValue">[docs]</a> <span class="k">def</span> <span class="nf">countByValue</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the count of each unique value in this RDD as a dictionary of</span>
<span class="sd"> (value, count) pairs.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sorted(sc.parallelize([1, 2, 1, 2, 2], 2).countByValue().items())</span>
<span class="sd"> [(1, 2), (2, 3)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">countPartition</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">counts</span> <span class="o">=</span> <span class="n">defaultdict</span><span class="p">(</span><span class="nb">int</span><span class="p">)</span>
<span class="k">for</span> <span class="n">obj</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="n">counts</span><span class="p">[</span><span class="n">obj</span><span class="p">]</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="k">yield</span> <span class="n">counts</span>
<span class="k">def</span> <span class="nf">mergeMaps</span><span class="p">(</span><span class="n">m1</span><span class="p">,</span> <span class="n">m2</span><span class="p">):</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">m2</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">m1</span><span class="p">[</span><span class="n">k</span><span class="p">]</span> <span class="o">+=</span> <span class="n">v</span>
<span class="k">return</span> <span class="n">m1</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">countPartition</span><span class="p">)</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="n">mergeMaps</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.top"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.top.html#pyspark.RDD.top">[docs]</a> <span class="k">def</span> <span class="nf">top</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">num</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Get the top N elements from an RDD.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This method should only be used if the resulting array is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> It returns the list sorted in descending order.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([10, 4, 2, 12, 3]).top(1)</span>
<span class="sd"> [12]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([2, 3, 4, 5, 6], 2).top(2)</span>
<span class="sd"> [6, 5]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([10, 4, 2, 12, 3]).top(3, key=str)</span>
<span class="sd"> [4, 3, 2]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">topIterator</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">heapq</span><span class="o">.</span><span class="n">nlargest</span><span class="p">(</span><span class="n">num</span><span class="p">,</span> <span class="n">iterator</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="n">key</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">merge</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">):</span>
<span class="k">return</span> <span class="n">heapq</span><span class="o">.</span><span class="n">nlargest</span><span class="p">(</span><span class="n">num</span><span class="p">,</span> <span class="n">a</span> <span class="o">+</span> <span class="n">b</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="n">key</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">topIterator</span><span class="p">)</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="n">merge</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.takeOrdered"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.takeOrdered.html#pyspark.RDD.takeOrdered">[docs]</a> <span class="k">def</span> <span class="nf">takeOrdered</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">num</span><span class="p">,</span> <span class="n">key</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Get the N elements from an RDD ordered in ascending order or as</span>
<span class="sd"> specified by the optional key function.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This method should only be used if the resulting array is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([10, 1, 2, 9, 3, 4, 5, 6, 7]).takeOrdered(6)</span>
<span class="sd"> [1, 2, 3, 4, 5, 6]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([10, 1, 2, 9, 3, 4, 5, 6, 7], 2).takeOrdered(6, key=lambda x: -x)</span>
<span class="sd"> [10, 9, 7, 6, 5, 4]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">merge</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">):</span>
<span class="k">return</span> <span class="n">heapq</span><span class="o">.</span><span class="n">nsmallest</span><span class="p">(</span><span class="n">num</span><span class="p">,</span> <span class="n">a</span> <span class="o">+</span> <span class="n">b</span><span class="p">,</span> <span class="n">key</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="k">lambda</span> <span class="n">it</span><span class="p">:</span> <span class="p">[</span><span class="n">heapq</span><span class="o">.</span><span class="n">nsmallest</span><span class="p">(</span><span class="n">num</span><span class="p">,</span> <span class="n">it</span><span class="p">,</span> <span class="n">key</span><span class="p">)])</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="n">merge</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.take"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.take.html#pyspark.RDD.take">[docs]</a> <span class="k">def</span> <span class="nf">take</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">num</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Take the first num elements of the RDD.</span>
<span class="sd"> It works by first scanning one partition, and use the results from</span>
<span class="sd"> that partition to estimate the number of additional partitions needed</span>
<span class="sd"> to satisfy the limit.</span>
<span class="sd"> Translated from the Scala implementation in RDD#take().</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This method should only be used if the resulting array is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([2, 3, 4, 5, 6]).cache().take(2)</span>
<span class="sd"> [2, 3]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([2, 3, 4, 5, 6]).take(10)</span>
<span class="sd"> [2, 3, 4, 5, 6]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize(range(100), 100).filter(lambda x: x &gt; 90).take(3)</span>
<span class="sd"> [91, 92, 93]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">items</span> <span class="o">=</span> <span class="p">[]</span>
<span class="n">totalParts</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span>
<span class="n">partsScanned</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">while</span> <span class="nb">len</span><span class="p">(</span><span class="n">items</span><span class="p">)</span> <span class="o">&lt;</span> <span class="n">num</span> <span class="ow">and</span> <span class="n">partsScanned</span> <span class="o">&lt;</span> <span class="n">totalParts</span><span class="p">:</span>
<span class="c1"># The number of partitions to try in this iteration.</span>
<span class="c1"># It is ok for this number to be greater than totalParts because</span>
<span class="c1"># we actually cap it at totalParts in runJob.</span>
<span class="n">numPartsToTry</span> <span class="o">=</span> <span class="mi">1</span>
<span class="k">if</span> <span class="n">partsScanned</span> <span class="o">&gt;</span> <span class="mi">0</span><span class="p">:</span>
<span class="c1"># If we didn&#39;t find any rows after the previous iteration,</span>
<span class="c1"># quadruple and retry. Otherwise, interpolate the number of</span>
<span class="c1"># partitions we need to try, but overestimate it by 50%.</span>
<span class="c1"># We also cap the estimation in the end.</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">items</span><span class="p">)</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">numPartsToTry</span> <span class="o">=</span> <span class="n">partsScanned</span> <span class="o">*</span> <span class="mi">4</span>
<span class="k">else</span><span class="p">:</span>
<span class="c1"># the first parameter of max is &gt;=1 whenever partsScanned &gt;= 2</span>
<span class="n">numPartsToTry</span> <span class="o">=</span> <span class="nb">int</span><span class="p">(</span><span class="mf">1.5</span> <span class="o">*</span> <span class="n">num</span> <span class="o">*</span> <span class="n">partsScanned</span> <span class="o">/</span> <span class="nb">len</span><span class="p">(</span><span class="n">items</span><span class="p">))</span> <span class="o">-</span> <span class="n">partsScanned</span>
<span class="n">numPartsToTry</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="nb">max</span><span class="p">(</span><span class="n">numPartsToTry</span><span class="p">,</span> <span class="mi">1</span><span class="p">),</span> <span class="n">partsScanned</span> <span class="o">*</span> <span class="mi">4</span><span class="p">)</span>
<span class="n">left</span> <span class="o">=</span> <span class="n">num</span> <span class="o">-</span> <span class="nb">len</span><span class="p">(</span><span class="n">items</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">takeUpToNumLeft</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">iterator</span> <span class="o">=</span> <span class="nb">iter</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="n">taken</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">while</span> <span class="n">taken</span> <span class="o">&lt;</span> <span class="n">left</span><span class="p">:</span>
<span class="k">try</span><span class="p">:</span>
<span class="k">yield</span> <span class="nb">next</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">except</span> <span class="ne">StopIteration</span><span class="p">:</span>
<span class="k">return</span>
<span class="n">taken</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="n">p</span> <span class="o">=</span> <span class="nb">range</span><span class="p">(</span><span class="n">partsScanned</span><span class="p">,</span> <span class="nb">min</span><span class="p">(</span><span class="n">partsScanned</span> <span class="o">+</span> <span class="n">numPartsToTry</span><span class="p">,</span> <span class="n">totalParts</span><span class="p">))</span>
<span class="n">res</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">context</span><span class="o">.</span><span class="n">runJob</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">takeUpToNumLeft</span><span class="p">,</span> <span class="n">p</span><span class="p">)</span>
<span class="n">items</span> <span class="o">+=</span> <span class="n">res</span>
<span class="n">partsScanned</span> <span class="o">+=</span> <span class="n">numPartsToTry</span>
<span class="k">return</span> <span class="n">items</span><span class="p">[:</span><span class="n">num</span><span class="p">]</span></div>
<div class="viewcode-block" id="RDD.first"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.first.html#pyspark.RDD.first">[docs]</a> <span class="k">def</span> <span class="nf">first</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the first element in this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([2, 3, 4]).first()</span>
<span class="sd"> 2</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([]).first()</span>
<span class="sd"> Traceback (most recent call last):</span>
<span class="sd"> ...</span>
<span class="sd"> ValueError: RDD is empty</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">rs</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">take</span><span class="p">(</span><span class="mi">1</span><span class="p">)</span>
<span class="k">if</span> <span class="n">rs</span><span class="p">:</span>
<span class="k">return</span> <span class="n">rs</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;RDD is empty&quot;</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.isEmpty"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.isEmpty.html#pyspark.RDD.isEmpty">[docs]</a> <span class="k">def</span> <span class="nf">isEmpty</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns true if and only if the RDD contains no elements at all.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> An RDD may be empty even when it has at least 1 partition.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([]).isEmpty()</span>
<span class="sd"> True</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1]).isEmpty()</span>
<span class="sd"> False</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span> <span class="o">==</span> <span class="mi">0</span> <span class="ow">or</span> <span class="nb">len</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">take</span><span class="p">(</span><span class="mi">1</span><span class="p">))</span> <span class="o">==</span> <span class="mi">0</span></div>
<div class="viewcode-block" id="RDD.saveAsNewAPIHadoopDataset"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.saveAsNewAPIHadoopDataset.html#pyspark.RDD.saveAsNewAPIHadoopDataset">[docs]</a> <span class="k">def</span> <span class="nf">saveAsNewAPIHadoopDataset</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">conf</span><span class="p">,</span> <span class="n">keyConverter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">valueConverter</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file</span>
<span class="sd"> system, using the new Hadoop OutputFormat API (mapreduce package). Keys/values are</span>
<span class="sd"> converted for output using either user specified converters or, by default,</span>
<span class="sd"> &quot;org.apache.spark.api.python.JavaToWritableConverter&quot;.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> conf : dict</span>
<span class="sd"> Hadoop job configuration</span>
<span class="sd"> keyConverter : str, optional</span>
<span class="sd"> fully qualified classname of key converter (None by default)</span>
<span class="sd"> valueConverter : str, optional</span>
<span class="sd"> fully qualified classname of value converter (None by default)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">jconf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_dictToJavaMap</span><span class="p">(</span><span class="n">conf</span><span class="p">)</span>
<span class="n">pickledRDD</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pickled</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">saveAsHadoopDataset</span><span class="p">(</span><span class="n">pickledRDD</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">,</span> <span class="kc">True</span><span class="p">,</span> <span class="n">jconf</span><span class="p">,</span>
<span class="n">keyConverter</span><span class="p">,</span> <span class="n">valueConverter</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.saveAsNewAPIHadoopFile"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.saveAsNewAPIHadoopFile.html#pyspark.RDD.saveAsNewAPIHadoopFile">[docs]</a> <span class="k">def</span> <span class="nf">saveAsNewAPIHadoopFile</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">outputFormatClass</span><span class="p">,</span> <span class="n">keyClass</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">valueClass</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">keyConverter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">valueConverter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">conf</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file</span>
<span class="sd"> system, using the new Hadoop OutputFormat API (mapreduce package). Key and value types</span>
<span class="sd"> will be inferred if not specified. Keys and values are converted for output using either</span>
<span class="sd"> user specified converters or &quot;org.apache.spark.api.python.JavaToWritableConverter&quot;. The</span>
<span class="sd"> `conf` is applied on top of the base Hadoop conf associated with the SparkContext</span>
<span class="sd"> of this RDD to create a merged Hadoop MapReduce job configuration for saving the data.</span>
<span class="sd"> path : str</span>
<span class="sd"> path to Hadoop file</span>
<span class="sd"> outputFormatClass : str</span>
<span class="sd"> fully qualified classname of Hadoop OutputFormat</span>
<span class="sd"> (e.g. &quot;org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat&quot;)</span>
<span class="sd"> keyClass : str, optional</span>
<span class="sd"> fully qualified classname of key Writable class</span>
<span class="sd"> (e.g. &quot;org.apache.hadoop.io.IntWritable&quot;, None by default)</span>
<span class="sd"> valueClass : str, optional</span>
<span class="sd"> fully qualified classname of value Writable class</span>
<span class="sd"> (e.g. &quot;org.apache.hadoop.io.Text&quot;, None by default)</span>
<span class="sd"> keyConverter : str, optional</span>
<span class="sd"> fully qualified classname of key converter (None by default)</span>
<span class="sd"> valueConverter : str, optional</span>
<span class="sd"> fully qualified classname of value converter (None by default)</span>
<span class="sd"> conf : dict, optional</span>
<span class="sd"> Hadoop job configuration (None by default)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">jconf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_dictToJavaMap</span><span class="p">(</span><span class="n">conf</span><span class="p">)</span>
<span class="n">pickledRDD</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pickled</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">saveAsNewAPIHadoopFile</span><span class="p">(</span><span class="n">pickledRDD</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">,</span> <span class="kc">True</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span>
<span class="n">outputFormatClass</span><span class="p">,</span>
<span class="n">keyClass</span><span class="p">,</span> <span class="n">valueClass</span><span class="p">,</span>
<span class="n">keyConverter</span><span class="p">,</span> <span class="n">valueConverter</span><span class="p">,</span> <span class="n">jconf</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.saveAsHadoopDataset"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.saveAsHadoopDataset.html#pyspark.RDD.saveAsHadoopDataset">[docs]</a> <span class="k">def</span> <span class="nf">saveAsHadoopDataset</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">conf</span><span class="p">,</span> <span class="n">keyConverter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">valueConverter</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file</span>
<span class="sd"> system, using the old Hadoop OutputFormat API (mapred package). Keys/values are</span>
<span class="sd"> converted for output using either user specified converters or, by default,</span>
<span class="sd"> &quot;org.apache.spark.api.python.JavaToWritableConverter&quot;.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> conf : dict</span>
<span class="sd"> Hadoop job configuration</span>
<span class="sd"> keyConverter : str, optional</span>
<span class="sd"> fully qualified classname of key converter (None by default)</span>
<span class="sd"> valueConverter : str, optional</span>
<span class="sd"> fully qualified classname of value converter (None by default)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">jconf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_dictToJavaMap</span><span class="p">(</span><span class="n">conf</span><span class="p">)</span>
<span class="n">pickledRDD</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pickled</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">saveAsHadoopDataset</span><span class="p">(</span><span class="n">pickledRDD</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">,</span> <span class="kc">True</span><span class="p">,</span> <span class="n">jconf</span><span class="p">,</span>
<span class="n">keyConverter</span><span class="p">,</span> <span class="n">valueConverter</span><span class="p">,</span> <span class="kc">False</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.saveAsHadoopFile"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.saveAsHadoopFile.html#pyspark.RDD.saveAsHadoopFile">[docs]</a> <span class="k">def</span> <span class="nf">saveAsHadoopFile</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">outputFormatClass</span><span class="p">,</span> <span class="n">keyClass</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">valueClass</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">keyConverter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">valueConverter</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">conf</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">compressionCodecClass</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file</span>
<span class="sd"> system, using the old Hadoop OutputFormat API (mapred package). Key and value types</span>
<span class="sd"> will be inferred if not specified. Keys and values are converted for output using either</span>
<span class="sd"> user specified converters or &quot;org.apache.spark.api.python.JavaToWritableConverter&quot;. The</span>
<span class="sd"> `conf` is applied on top of the base Hadoop conf associated with the SparkContext</span>
<span class="sd"> of this RDD to create a merged Hadoop MapReduce job configuration for saving the data.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> path : str</span>
<span class="sd"> path to Hadoop file</span>
<span class="sd"> outputFormatClass : str</span>
<span class="sd"> fully qualified classname of Hadoop OutputFormat</span>
<span class="sd"> (e.g. &quot;org.apache.hadoop.mapred.SequenceFileOutputFormat&quot;)</span>
<span class="sd"> keyClass : str, optional</span>
<span class="sd"> fully qualified classname of key Writable class</span>
<span class="sd"> (e.g. &quot;org.apache.hadoop.io.IntWritable&quot;, None by default)</span>
<span class="sd"> valueClass : str, optional</span>
<span class="sd"> fully qualified classname of value Writable class</span>
<span class="sd"> (e.g. &quot;org.apache.hadoop.io.Text&quot;, None by default)</span>
<span class="sd"> keyConverter : str, optional</span>
<span class="sd"> fully qualified classname of key converter (None by default)</span>
<span class="sd"> valueConverter : str, optional</span>
<span class="sd"> fully qualified classname of value converter (None by default)</span>
<span class="sd"> conf : dict, optional</span>
<span class="sd"> (None by default)</span>
<span class="sd"> compressionCodecClass : str</span>
<span class="sd"> fully qualified classname of the compression codec class</span>
<span class="sd"> i.e. &quot;org.apache.hadoop.io.compress.GzipCodec&quot; (None by default)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">jconf</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_dictToJavaMap</span><span class="p">(</span><span class="n">conf</span><span class="p">)</span>
<span class="n">pickledRDD</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pickled</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">saveAsHadoopFile</span><span class="p">(</span><span class="n">pickledRDD</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">,</span> <span class="kc">True</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span>
<span class="n">outputFormatClass</span><span class="p">,</span>
<span class="n">keyClass</span><span class="p">,</span> <span class="n">valueClass</span><span class="p">,</span>
<span class="n">keyConverter</span><span class="p">,</span> <span class="n">valueConverter</span><span class="p">,</span>
<span class="n">jconf</span><span class="p">,</span> <span class="n">compressionCodecClass</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.saveAsSequenceFile"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.saveAsSequenceFile.html#pyspark.RDD.saveAsSequenceFile">[docs]</a> <span class="k">def</span> <span class="nf">saveAsSequenceFile</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">compressionCodecClass</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Output a Python RDD of key-value pairs (of form ``RDD[(K, V)]``) to any Hadoop file</span>
<span class="sd"> system, using the &quot;org.apache.hadoop.io.Writable&quot; types that we convert from the</span>
<span class="sd"> RDD&#39;s key and value types. The mechanism is as follows:</span>
<span class="sd"> 1. Pyrolite is used to convert pickled Python RDD into RDD of Java objects.</span>
<span class="sd"> 2. Keys and values of this Java RDD are converted to Writables and written out.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> path : str</span>
<span class="sd"> path to sequence file</span>
<span class="sd"> compressionCodecClass : str, optional</span>
<span class="sd"> fully qualified classname of the compression codec class</span>
<span class="sd"> i.e. &quot;org.apache.hadoop.io.compress.GzipCodec&quot; (None by default)</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">pickledRDD</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pickled</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">saveAsSequenceFile</span><span class="p">(</span><span class="n">pickledRDD</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">,</span> <span class="kc">True</span><span class="p">,</span>
<span class="n">path</span><span class="p">,</span> <span class="n">compressionCodecClass</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.saveAsPickleFile"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.saveAsPickleFile.html#pyspark.RDD.saveAsPickleFile">[docs]</a> <span class="k">def</span> <span class="nf">saveAsPickleFile</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">batchSize</span><span class="o">=</span><span class="mi">10</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Save this RDD as a SequenceFile of serialized objects. The serializer</span>
<span class="sd"> used is :class:`pyspark.serializers.PickleSerializer`, default batch size</span>
<span class="sd"> is 10.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from tempfile import NamedTemporaryFile</span>
<span class="sd"> &gt;&gt;&gt; tmpFile = NamedTemporaryFile(delete=True)</span>
<span class="sd"> &gt;&gt;&gt; tmpFile.close()</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, &#39;spark&#39;, &#39;rdd&#39;]).saveAsPickleFile(tmpFile.name, 3)</span>
<span class="sd"> &gt;&gt;&gt; sorted(sc.pickleFile(tmpFile.name, 5).map(str).collect())</span>
<span class="sd"> [&#39;1&#39;, &#39;2&#39;, &#39;rdd&#39;, &#39;spark&#39;]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">batchSize</span> <span class="o">==</span> <span class="mi">0</span><span class="p">:</span>
<span class="n">ser</span> <span class="o">=</span> <span class="n">AutoBatchedSerializer</span><span class="p">(</span><span class="n">PickleSerializer</span><span class="p">())</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">ser</span> <span class="o">=</span> <span class="n">BatchedSerializer</span><span class="p">(</span><span class="n">PickleSerializer</span><span class="p">(),</span> <span class="n">batchSize</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_reserialize</span><span class="p">(</span><span class="n">ser</span><span class="p">)</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">saveAsObjectFile</span><span class="p">(</span><span class="n">path</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.saveAsTextFile"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.saveAsTextFile.html#pyspark.RDD.saveAsTextFile">[docs]</a> <span class="k">def</span> <span class="nf">saveAsTextFile</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">path</span><span class="p">,</span> <span class="n">compressionCodecClass</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Save this RDD as a text file, using string representations of elements.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> path : str</span>
<span class="sd"> path to text file</span>
<span class="sd"> compressionCodecClass : str, optional</span>
<span class="sd"> fully qualified classname of the compression codec class</span>
<span class="sd"> i.e. &quot;org.apache.hadoop.io.compress.GzipCodec&quot; (None by default)</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from tempfile import NamedTemporaryFile</span>
<span class="sd"> &gt;&gt;&gt; tempFile = NamedTemporaryFile(delete=True)</span>
<span class="sd"> &gt;&gt;&gt; tempFile.close()</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize(range(10)).saveAsTextFile(tempFile.name)</span>
<span class="sd"> &gt;&gt;&gt; from fileinput import input</span>
<span class="sd"> &gt;&gt;&gt; from glob import glob</span>
<span class="sd"> &gt;&gt;&gt; &#39;&#39;.join(sorted(input(glob(tempFile.name + &quot;/part-0000*&quot;))))</span>
<span class="sd"> &#39;0\\n1\\n2\\n3\\n4\\n5\\n6\\n7\\n8\\n9\\n&#39;</span>
<span class="sd"> Empty lines are tolerated when saving to text files.</span>
<span class="sd"> &gt;&gt;&gt; from tempfile import NamedTemporaryFile</span>
<span class="sd"> &gt;&gt;&gt; tempFile2 = NamedTemporaryFile(delete=True)</span>
<span class="sd"> &gt;&gt;&gt; tempFile2.close()</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([&#39;&#39;, &#39;foo&#39;, &#39;&#39;, &#39;bar&#39;, &#39;&#39;]).saveAsTextFile(tempFile2.name)</span>
<span class="sd"> &gt;&gt;&gt; &#39;&#39;.join(sorted(input(glob(tempFile2.name + &quot;/part-0000*&quot;))))</span>
<span class="sd"> &#39;\\n\\n\\nbar\\nfoo\\n&#39;</span>
<span class="sd"> Using compressionCodecClass</span>
<span class="sd"> &gt;&gt;&gt; from tempfile import NamedTemporaryFile</span>
<span class="sd"> &gt;&gt;&gt; tempFile3 = NamedTemporaryFile(delete=True)</span>
<span class="sd"> &gt;&gt;&gt; tempFile3.close()</span>
<span class="sd"> &gt;&gt;&gt; codec = &quot;org.apache.hadoop.io.compress.GzipCodec&quot;</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([&#39;foo&#39;, &#39;bar&#39;]).saveAsTextFile(tempFile3.name, codec)</span>
<span class="sd"> &gt;&gt;&gt; from fileinput import input, hook_compressed</span>
<span class="sd"> &gt;&gt;&gt; result = sorted(input(glob(tempFile3.name + &quot;/part*.gz&quot;), openhook=hook_compressed))</span>
<span class="sd"> &gt;&gt;&gt; b&#39;&#39;.join(result).decode(&#39;utf-8&#39;)</span>
<span class="sd"> &#39;bar\\nfoo\\n&#39;</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">split</span><span class="p">,</span> <span class="n">iterator</span><span class="p">):</span>
<span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">iterator</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">x</span><span class="p">,</span> <span class="p">(</span><span class="nb">str</span><span class="p">,</span> <span class="nb">bytes</span><span class="p">)):</span>
<span class="n">x</span> <span class="o">=</span> <span class="nb">str</span><span class="p">(</span><span class="n">x</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="nb">str</span><span class="p">):</span>
<span class="n">x</span> <span class="o">=</span> <span class="n">x</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="s2">&quot;utf-8&quot;</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">x</span>
<span class="n">keyed</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">func</span><span class="p">)</span>
<span class="n">keyed</span><span class="o">.</span><span class="n">_bypass_serializer</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">if</span> <span class="n">compressionCodecClass</span><span class="p">:</span>
<span class="n">compressionCodec</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">java</span><span class="o">.</span><span class="n">lang</span><span class="o">.</span><span class="n">Class</span><span class="o">.</span><span class="n">forName</span><span class="p">(</span><span class="n">compressionCodecClass</span><span class="p">)</span>
<span class="n">keyed</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">BytesToString</span><span class="p">())</span><span class="o">.</span><span class="n">saveAsTextFile</span><span class="p">(</span><span class="n">path</span><span class="p">,</span> <span class="n">compressionCodec</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">keyed</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">BytesToString</span><span class="p">())</span><span class="o">.</span><span class="n">saveAsTextFile</span><span class="p">(</span><span class="n">path</span><span class="p">)</span></div>
<span class="c1"># Pair functions</span>
<div class="viewcode-block" id="RDD.collectAsMap"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.collectAsMap.html#pyspark.RDD.collectAsMap">[docs]</a> <span class="k">def</span> <span class="nf">collectAsMap</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the key-value pairs in this RDD to the master as a dictionary.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This method should only be used if the resulting data is expected</span>
<span class="sd"> to be small, as all the data is loaded into the driver&#39;s memory.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; m = sc.parallelize([(1, 2), (3, 4)]).collectAsMap()</span>
<span class="sd"> &gt;&gt;&gt; m[1]</span>
<span class="sd"> 2</span>
<span class="sd"> &gt;&gt;&gt; m[3]</span>
<span class="sd"> 4</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="nb">dict</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">collect</span><span class="p">())</span></div>
<div class="viewcode-block" id="RDD.keys"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.keys.html#pyspark.RDD.keys">[docs]</a> <span class="k">def</span> <span class="nf">keys</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return an RDD with the keys of each tuple.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; m = sc.parallelize([(1, 2), (3, 4)]).keys()</span>
<span class="sd"> &gt;&gt;&gt; m.collect()</span>
<span class="sd"> [1, 3]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span></div>
<div class="viewcode-block" id="RDD.values"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.values.html#pyspark.RDD.values">[docs]</a> <span class="k">def</span> <span class="nf">values</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return an RDD with the values of each tuple.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; m = sc.parallelize([(1, 2), (3, 4)]).values()</span>
<span class="sd"> &gt;&gt;&gt; m.collect()</span>
<span class="sd"> [2, 4]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">[</span><span class="mi">1</span><span class="p">])</span></div>
<div class="viewcode-block" id="RDD.reduceByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.reduceByKey.html#pyspark.RDD.reduceByKey">[docs]</a> <span class="k">def</span> <span class="nf">reduceByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="o">=</span><span class="n">portable_hash</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Merge the values for each key using an associative and commutative reduce function.</span>
<span class="sd"> This will also perform the merging locally on each mapper before</span>
<span class="sd"> sending results to a reducer, similarly to a &quot;combiner&quot; in MapReduce.</span>
<span class="sd"> Output will be partitioned with `numPartitions` partitions, or</span>
<span class="sd"> the default parallelism level if `numPartitions` is not specified.</span>
<span class="sd"> Default partitioner is hash-partition.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from operator import add</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 1), (&quot;a&quot;, 1)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.reduceByKey(add).collect())</span>
<span class="sd"> [(&#39;a&#39;, 2), (&#39;b&#39;, 1)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combineByKey</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.reduceByKeyLocally"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.reduceByKeyLocally.html#pyspark.RDD.reduceByKeyLocally">[docs]</a> <span class="k">def</span> <span class="nf">reduceByKeyLocally</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">func</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Merge the values for each key using an associative and commutative reduce function, but</span>
<span class="sd"> return the results immediately to the master as a dictionary.</span>
<span class="sd"> This will also perform the merging locally on each mapper before</span>
<span class="sd"> sending results to a reducer, similarly to a &quot;combiner&quot; in MapReduce.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; from operator import add</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 1), (&quot;a&quot;, 1)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.reduceByKeyLocally(add).items())</span>
<span class="sd"> [(&#39;a&#39;, 2), (&#39;b&#39;, 1)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">func</span> <span class="o">=</span> <span class="n">fail_on_stopiteration</span><span class="p">(</span><span class="n">func</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">reducePartition</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">m</span> <span class="o">=</span> <span class="p">{}</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="n">m</span><span class="p">[</span><span class="n">k</span><span class="p">]</span> <span class="o">=</span> <span class="n">func</span><span class="p">(</span><span class="n">m</span><span class="p">[</span><span class="n">k</span><span class="p">],</span> <span class="n">v</span><span class="p">)</span> <span class="k">if</span> <span class="n">k</span> <span class="ow">in</span> <span class="n">m</span> <span class="k">else</span> <span class="n">v</span>
<span class="k">yield</span> <span class="n">m</span>
<span class="k">def</span> <span class="nf">mergeMaps</span><span class="p">(</span><span class="n">m1</span><span class="p">,</span> <span class="n">m2</span><span class="p">):</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">m2</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="n">m1</span><span class="p">[</span><span class="n">k</span><span class="p">]</span> <span class="o">=</span> <span class="n">func</span><span class="p">(</span><span class="n">m1</span><span class="p">[</span><span class="n">k</span><span class="p">],</span> <span class="n">v</span><span class="p">)</span> <span class="k">if</span> <span class="n">k</span> <span class="ow">in</span> <span class="n">m1</span> <span class="k">else</span> <span class="n">v</span>
<span class="k">return</span> <span class="n">m1</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">reducePartition</span><span class="p">)</span><span class="o">.</span><span class="n">reduce</span><span class="p">(</span><span class="n">mergeMaps</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.countByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.countByKey.html#pyspark.RDD.countByKey">[docs]</a> <span class="k">def</span> <span class="nf">countByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Count the number of elements for each key, and return the result to the</span>
<span class="sd"> master as a dictionary.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 1), (&quot;a&quot;, 1)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.countByKey().items())</span>
<span class="sd"> [(&#39;a&#39;, 2), (&#39;b&#39;, 1)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span><span class="o">.</span><span class="n">countByValue</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.join"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.join.html#pyspark.RDD.join">[docs]</a> <span class="k">def</span> <span class="nf">join</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">other</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return an RDD containing all pairs of elements with matching keys in</span>
<span class="sd"> `self` and `other`.</span>
<span class="sd"> Each pair of elements will be returned as a (k, (v1, v2)) tuple, where</span>
<span class="sd"> (k, v1) is in `self` and (k, v2) is in `other`.</span>
<span class="sd"> Performs a hash join across the cluster.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 4)])</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize([(&quot;a&quot;, 2), (&quot;a&quot;, 3)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(x.join(y).collect())</span>
<span class="sd"> [(&#39;a&#39;, (1, 2)), (&#39;a&#39;, (1, 3))]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">python_join</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">numPartitions</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.leftOuterJoin"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.leftOuterJoin.html#pyspark.RDD.leftOuterJoin">[docs]</a> <span class="k">def</span> <span class="nf">leftOuterJoin</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">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Perform a left outer join of `self` and `other`.</span>
<span class="sd"> For each element (k, v) in `self`, the resulting RDD will either</span>
<span class="sd"> contain all pairs (k, (v, w)) for w in `other`, or the pair</span>
<span class="sd"> (k, (v, None)) if no elements in `other` have key k.</span>
<span class="sd"> Hash-partitions the resulting RDD into the given number of partitions.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 4)])</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize([(&quot;a&quot;, 2)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(x.leftOuterJoin(y).collect())</span>
<span class="sd"> [(&#39;a&#39;, (1, 2)), (&#39;b&#39;, (4, None))]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">python_left_outer_join</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">numPartitions</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.rightOuterJoin"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.rightOuterJoin.html#pyspark.RDD.rightOuterJoin">[docs]</a> <span class="k">def</span> <span class="nf">rightOuterJoin</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">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Perform a right outer join of `self` and `other`.</span>
<span class="sd"> For each element (k, w) in `other`, the resulting RDD will either</span>
<span class="sd"> contain all pairs (k, (v, w)) for v in this, or the pair (k, (None, w))</span>
<span class="sd"> if no elements in `self` have key k.</span>
<span class="sd"> Hash-partitions the resulting RDD into the given number of partitions.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 4)])</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize([(&quot;a&quot;, 2)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(y.rightOuterJoin(x).collect())</span>
<span class="sd"> [(&#39;a&#39;, (2, 1)), (&#39;b&#39;, (None, 4))]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">python_right_outer_join</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">numPartitions</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.fullOuterJoin"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.fullOuterJoin.html#pyspark.RDD.fullOuterJoin">[docs]</a> <span class="k">def</span> <span class="nf">fullOuterJoin</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">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Perform a right outer join of `self` and `other`.</span>
<span class="sd"> For each element (k, v) in `self`, the resulting RDD will either</span>
<span class="sd"> contain all pairs (k, (v, w)) for w in `other`, or the pair</span>
<span class="sd"> (k, (v, None)) if no elements in `other` have key k.</span>
<span class="sd"> Similarly, for each element (k, w) in `other`, the resulting RDD will</span>
<span class="sd"> either contain all pairs (k, (v, w)) for v in `self`, or the pair</span>
<span class="sd"> (k, (None, w)) if no elements in `self` have key k.</span>
<span class="sd"> Hash-partitions the resulting RDD into the given number of partitions.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 4)])</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize([(&quot;a&quot;, 2), (&quot;c&quot;, 8)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(x.fullOuterJoin(y).collect())</span>
<span class="sd"> [(&#39;a&#39;, (1, 2)), (&#39;b&#39;, (4, None)), (&#39;c&#39;, (None, 8))]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">python_full_outer_join</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">numPartitions</span><span class="p">)</span></div>
<span class="c1"># TODO: add option to control map-side combining</span>
<span class="c1"># portable_hash is used as default, because builtin hash of None is different</span>
<span class="c1"># cross machines.</span>
<div class="viewcode-block" id="RDD.partitionBy"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.partitionBy.html#pyspark.RDD.partitionBy">[docs]</a> <span class="k">def</span> <span class="nf">partitionBy</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="o">=</span><span class="n">portable_hash</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a copy of the RDD partitioned using the specified partitioner.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; pairs = sc.parallelize([1, 2, 3, 4, 2, 4, 1]).map(lambda x: (x, x))</span>
<span class="sd"> &gt;&gt;&gt; sets = pairs.partitionBy(2).glom().collect()</span>
<span class="sd"> &gt;&gt;&gt; len(set(sets[0]).intersection(set(sets[1])))</span>
<span class="sd"> 0</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">numPartitions</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">numPartitions</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_defaultReducePartitions</span><span class="p">()</span>
<span class="n">partitioner</span> <span class="o">=</span> <span class="n">Partitioner</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">partitioner</span> <span class="o">==</span> <span class="n">partitioner</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span>
<span class="c1"># Transferring O(n) objects to Java is too expensive.</span>
<span class="c1"># Instead, we&#39;ll form the hash buckets in Python,</span>
<span class="c1"># transferring O(numPartitions) objects to Java.</span>
<span class="c1"># Each object is a (splitNumber, [objects]) pair.</span>
<span class="c1"># In order to avoid too huge objects, the objects are</span>
<span class="c1"># grouped into chunks.</span>
<span class="n">outputSerializer</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_unbatched_serializer</span>
<span class="n">limit</span> <span class="o">=</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_memory_limit</span><span class="p">()</span> <span class="o">/</span> <span class="mi">2</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">add_shuffle_key</span><span class="p">(</span><span class="n">split</span><span class="p">,</span> <span class="n">iterator</span><span class="p">):</span>
<span class="n">buckets</span> <span class="o">=</span> <span class="n">defaultdict</span><span class="p">(</span><span class="nb">list</span><span class="p">)</span>
<span class="n">c</span><span class="p">,</span> <span class="n">batch</span> <span class="o">=</span> <span class="mi">0</span><span class="p">,</span> <span class="nb">min</span><span class="p">(</span><span class="mi">10</span> <span class="o">*</span> <span class="n">numPartitions</span><span class="p">,</span> <span class="mi">1000</span><span class="p">)</span>
<span class="k">for</span> <span class="n">k</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="n">iterator</span><span class="p">:</span>
<span class="n">buckets</span><span class="p">[</span><span class="n">partitionFunc</span><span class="p">(</span><span class="n">k</span><span class="p">)</span> <span class="o">%</span> <span class="n">numPartitions</span><span class="p">]</span><span class="o">.</span><span class="n">append</span><span class="p">((</span><span class="n">k</span><span class="p">,</span> <span class="n">v</span><span class="p">))</span>
<span class="n">c</span> <span class="o">+=</span> <span class="mi">1</span>
<span class="c1"># check used memory and avg size of chunk of objects</span>
<span class="k">if</span> <span class="p">(</span><span class="n">c</span> <span class="o">%</span> <span class="mi">1000</span> <span class="o">==</span> <span class="mi">0</span> <span class="ow">and</span> <span class="n">get_used_memory</span><span class="p">()</span> <span class="o">&gt;</span> <span class="n">limit</span>
<span class="ow">or</span> <span class="n">c</span> <span class="o">&gt;</span> <span class="n">batch</span><span class="p">):</span>
<span class="n">n</span><span class="p">,</span> <span class="n">size</span> <span class="o">=</span> <span class="nb">len</span><span class="p">(</span><span class="n">buckets</span><span class="p">),</span> <span class="mi">0</span>
<span class="k">for</span> <span class="n">split</span> <span class="ow">in</span> <span class="nb">list</span><span class="p">(</span><span class="n">buckets</span><span class="o">.</span><span class="n">keys</span><span class="p">()):</span>
<span class="k">yield</span> <span class="n">pack_long</span><span class="p">(</span><span class="n">split</span><span class="p">)</span>
<span class="n">d</span> <span class="o">=</span> <span class="n">outputSerializer</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">buckets</span><span class="p">[</span><span class="n">split</span><span class="p">])</span>
<span class="k">del</span> <span class="n">buckets</span><span class="p">[</span><span class="n">split</span><span class="p">]</span>
<span class="k">yield</span> <span class="n">d</span>
<span class="n">size</span> <span class="o">+=</span> <span class="nb">len</span><span class="p">(</span><span class="n">d</span><span class="p">)</span>
<span class="n">avg</span> <span class="o">=</span> <span class="nb">int</span><span class="p">(</span><span class="n">size</span> <span class="o">/</span> <span class="n">n</span><span class="p">)</span> <span class="o">&gt;&gt;</span> <span class="mi">20</span>
<span class="c1"># let 1M &lt; avg &lt; 10M</span>
<span class="k">if</span> <span class="n">avg</span> <span class="o">&lt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">batch</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">sys</span><span class="o">.</span><span class="n">maxsize</span><span class="p">,</span> <span class="n">batch</span> <span class="o">*</span> <span class="mf">1.5</span><span class="p">)</span>
<span class="k">elif</span> <span class="n">avg</span> <span class="o">&gt;</span> <span class="mi">10</span><span class="p">:</span>
<span class="n">batch</span> <span class="o">=</span> <span class="nb">max</span><span class="p">(</span><span class="nb">int</span><span class="p">(</span><span class="n">batch</span> <span class="o">/</span> <span class="mf">1.5</span><span class="p">),</span> <span class="mi">1</span><span class="p">)</span>
<span class="n">c</span> <span class="o">=</span> <span class="mi">0</span>
<span class="k">for</span> <span class="n">split</span><span class="p">,</span> <span class="n">items</span> <span class="ow">in</span> <span class="n">buckets</span><span class="o">.</span><span class="n">items</span><span class="p">():</span>
<span class="k">yield</span> <span class="n">pack_long</span><span class="p">(</span><span class="n">split</span><span class="p">)</span>
<span class="k">yield</span> <span class="n">outputSerializer</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">items</span><span class="p">)</span>
<span class="n">keyed</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">add_shuffle_key</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">keyed</span><span class="o">.</span><span class="n">_bypass_serializer</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">with</span> <span class="n">SCCallSiteSync</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">context</span><span class="p">)</span> <span class="k">as</span> <span class="n">css</span><span class="p">:</span>
<span class="n">pairRDD</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PairwiseRDD</span><span class="p">(</span>
<span class="n">keyed</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">())</span><span class="o">.</span><span class="n">asJavaPairRDD</span><span class="p">()</span>
<span class="n">jpartitioner</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonPartitioner</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span>
<span class="nb">id</span><span class="p">(</span><span class="n">partitionFunc</span><span class="p">))</span>
<span class="n">jrdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">valueOfPair</span><span class="p">(</span><span class="n">pairRDD</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="n">jpartitioner</span><span class="p">))</span>
<span class="n">rdd</span> <span class="o">=</span> <span class="n">RDD</span><span class="p">(</span><span class="n">jrdd</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="p">,</span> <span class="n">BatchedSerializer</span><span class="p">(</span><span class="n">outputSerializer</span><span class="p">))</span>
<span class="n">rdd</span><span class="o">.</span><span class="n">partitioner</span> <span class="o">=</span> <span class="n">partitioner</span>
<span class="k">return</span> <span class="n">rdd</span></div>
<span class="c1"># TODO: add control over map-side aggregation</span>
<div class="viewcode-block" id="RDD.combineByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.combineByKey.html#pyspark.RDD.combineByKey">[docs]</a> <span class="k">def</span> <span class="nf">combineByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">createCombiner</span><span class="p">,</span> <span class="n">mergeValue</span><span class="p">,</span> <span class="n">mergeCombiners</span><span class="p">,</span>
<span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="o">=</span><span class="n">portable_hash</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Generic function to combine the elements for each key using a custom</span>
<span class="sd"> set of aggregation functions.</span>
<span class="sd"> Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a &quot;combined</span>
<span class="sd"> type&quot; C.</span>
<span class="sd"> Users provide three functions:</span>
<span class="sd"> - `createCombiner`, which turns a V into a C (e.g., creates</span>
<span class="sd"> a one-element list)</span>
<span class="sd"> - `mergeValue`, to merge a V into a C (e.g., adds it to the end of</span>
<span class="sd"> a list)</span>
<span class="sd"> - `mergeCombiners`, to combine two C&#39;s into a single one (e.g., merges</span>
<span class="sd"> the lists)</span>
<span class="sd"> To avoid memory allocation, both mergeValue and mergeCombiners are allowed to</span>
<span class="sd"> modify and return their first argument instead of creating a new C.</span>
<span class="sd"> In addition, users can control the partitioning of the output RDD.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> V and C can be different -- for example, one might group an RDD of type</span>
<span class="sd"> (Int, Int) into an RDD of type (Int, List[Int]).</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 1), (&quot;a&quot;, 2)])</span>
<span class="sd"> &gt;&gt;&gt; def to_list(a):</span>
<span class="sd"> ... return [a]</span>
<span class="sd"> ...</span>
<span class="sd"> &gt;&gt;&gt; def append(a, b):</span>
<span class="sd"> ... a.append(b)</span>
<span class="sd"> ... return a</span>
<span class="sd"> ...</span>
<span class="sd"> &gt;&gt;&gt; def extend(a, b):</span>
<span class="sd"> ... a.extend(b)</span>
<span class="sd"> ... return a</span>
<span class="sd"> ...</span>
<span class="sd"> &gt;&gt;&gt; sorted(x.combineByKey(to_list, append, extend).collect())</span>
<span class="sd"> [(&#39;a&#39;, [1, 2]), (&#39;b&#39;, [1])]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">numPartitions</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">numPartitions</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_defaultReducePartitions</span><span class="p">()</span>
<span class="n">serializer</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">serializer</span>
<span class="n">memory</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_memory_limit</span><span class="p">()</span>
<span class="n">agg</span> <span class="o">=</span> <span class="n">Aggregator</span><span class="p">(</span><span class="n">createCombiner</span><span class="p">,</span> <span class="n">mergeValue</span><span class="p">,</span> <span class="n">mergeCombiners</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">combineLocally</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">merger</span> <span class="o">=</span> <span class="n">ExternalMerger</span><span class="p">(</span><span class="n">agg</span><span class="p">,</span> <span class="n">memory</span> <span class="o">*</span> <span class="mf">0.9</span><span class="p">,</span> <span class="n">serializer</span><span class="p">)</span>
<span class="n">merger</span><span class="o">.</span><span class="n">mergeValues</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">return</span> <span class="n">merger</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="n">locally_combined</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">combineLocally</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">shuffled</span> <span class="o">=</span> <span class="n">locally_combined</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_mergeCombiners</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">merger</span> <span class="o">=</span> <span class="n">ExternalMerger</span><span class="p">(</span><span class="n">agg</span><span class="p">,</span> <span class="n">memory</span><span class="p">,</span> <span class="n">serializer</span><span class="p">)</span>
<span class="n">merger</span><span class="o">.</span><span class="n">mergeCombiners</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">return</span> <span class="n">merger</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="k">return</span> <span class="n">shuffled</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">_mergeCombiners</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.aggregateByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.aggregateByKey.html#pyspark.RDD.aggregateByKey">[docs]</a> <span class="k">def</span> <span class="nf">aggregateByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">zeroValue</span><span class="p">,</span> <span class="n">seqFunc</span><span class="p">,</span> <span class="n">combFunc</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span>
<span class="n">partitionFunc</span><span class="o">=</span><span class="n">portable_hash</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Aggregate the values of each key, using given combine functions and a neutral</span>
<span class="sd"> &quot;zero value&quot;. This function can return a different result type, U, than the type</span>
<span class="sd"> of the values in this RDD, V. Thus, we need one operation for merging a V into</span>
<span class="sd"> a U and one operation for merging two U&#39;s, The former operation is used for merging</span>
<span class="sd"> values within a partition, and the latter is used for merging values between</span>
<span class="sd"> partitions. To avoid memory allocation, both of these functions are</span>
<span class="sd"> allowed to modify and return their first argument instead of creating a new U.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">createZero</span><span class="p">():</span>
<span class="k">return</span> <span class="n">copy</span><span class="o">.</span><span class="n">deepcopy</span><span class="p">(</span><span class="n">zeroValue</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combineByKey</span><span class="p">(</span>
<span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="n">seqFunc</span><span class="p">(</span><span class="n">createZero</span><span class="p">(),</span> <span class="n">v</span><span class="p">),</span> <span class="n">seqFunc</span><span class="p">,</span> <span class="n">combFunc</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.foldByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.foldByKey.html#pyspark.RDD.foldByKey">[docs]</a> <span class="k">def</span> <span class="nf">foldByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">zeroValue</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="o">=</span><span class="n">portable_hash</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Merge the values for each key using an associative function &quot;func&quot;</span>
<span class="sd"> and a neutral &quot;zeroValue&quot; which may be added to the result an</span>
<span class="sd"> arbitrary number of times, and must not change the result</span>
<span class="sd"> (e.g., 0 for addition, or 1 for multiplication.).</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 1), (&quot;a&quot;, 1)])</span>
<span class="sd"> &gt;&gt;&gt; from operator import add</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.foldByKey(0, add).collect())</span>
<span class="sd"> [(&#39;a&#39;, 2), (&#39;b&#39;, 1)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">createZero</span><span class="p">():</span>
<span class="k">return</span> <span class="n">copy</span><span class="o">.</span><span class="n">deepcopy</span><span class="p">(</span><span class="n">zeroValue</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">combineByKey</span><span class="p">(</span><span class="k">lambda</span> <span class="n">v</span><span class="p">:</span> <span class="n">func</span><span class="p">(</span><span class="n">createZero</span><span class="p">(),</span> <span class="n">v</span><span class="p">),</span> <span class="n">func</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">,</span>
<span class="n">partitionFunc</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_memory_limit</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="n">_parse_memory</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_conf</span><span class="o">.</span><span class="n">get</span><span class="p">(</span><span class="s2">&quot;spark.python.worker.memory&quot;</span><span class="p">,</span> <span class="s2">&quot;512m&quot;</span><span class="p">))</span>
<span class="c1"># TODO: support variant with custom partitioner</span>
<div class="viewcode-block" id="RDD.groupByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.groupByKey.html#pyspark.RDD.groupByKey">[docs]</a> <span class="k">def</span> <span class="nf">groupByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="o">=</span><span class="n">portable_hash</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Group the values for each key in the RDD into a single sequence.</span>
<span class="sd"> Hash-partitions the resulting RDD with numPartitions partitions.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> If you are grouping in order to perform an aggregation (such as a</span>
<span class="sd"> sum or average) over each key, using reduceByKey or aggregateByKey will</span>
<span class="sd"> provide much better performance.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 1), (&quot;a&quot;, 1)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.groupByKey().mapValues(len).collect())</span>
<span class="sd"> [(&#39;a&#39;, 2), (&#39;b&#39;, 1)]</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.groupByKey().mapValues(list).collect())</span>
<span class="sd"> [(&#39;a&#39;, [1, 1]), (&#39;b&#39;, [1])]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">createCombiner</span><span class="p">(</span><span class="n">x</span><span class="p">):</span>
<span class="k">return</span> <span class="p">[</span><span class="n">x</span><span class="p">]</span>
<span class="k">def</span> <span class="nf">mergeValue</span><span class="p">(</span><span class="n">xs</span><span class="p">,</span> <span class="n">x</span><span class="p">):</span>
<span class="n">xs</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">x</span><span class="p">)</span>
<span class="k">return</span> <span class="n">xs</span>
<span class="k">def</span> <span class="nf">mergeCombiners</span><span class="p">(</span><span class="n">a</span><span class="p">,</span> <span class="n">b</span><span class="p">):</span>
<span class="n">a</span><span class="o">.</span><span class="n">extend</span><span class="p">(</span><span class="n">b</span><span class="p">)</span>
<span class="k">return</span> <span class="n">a</span>
<span class="n">memory</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_memory_limit</span><span class="p">()</span>
<span class="n">serializer</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span>
<span class="n">agg</span> <span class="o">=</span> <span class="n">Aggregator</span><span class="p">(</span><span class="n">createCombiner</span><span class="p">,</span> <span class="n">mergeValue</span><span class="p">,</span> <span class="n">mergeCombiners</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">combine</span><span class="p">(</span><span class="n">iterator</span><span class="p">):</span>
<span class="n">merger</span> <span class="o">=</span> <span class="n">ExternalMerger</span><span class="p">(</span><span class="n">agg</span><span class="p">,</span> <span class="n">memory</span> <span class="o">*</span> <span class="mf">0.9</span><span class="p">,</span> <span class="n">serializer</span><span class="p">)</span>
<span class="n">merger</span><span class="o">.</span><span class="n">mergeValues</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">return</span> <span class="n">merger</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="n">locally_combined</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">combine</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span>
<span class="n">shuffled</span> <span class="o">=</span> <span class="n">locally_combined</span><span class="o">.</span><span class="n">partitionBy</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">partitionFunc</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">groupByKey</span><span class="p">(</span><span class="n">it</span><span class="p">):</span>
<span class="n">merger</span> <span class="o">=</span> <span class="n">ExternalGroupBy</span><span class="p">(</span><span class="n">agg</span><span class="p">,</span> <span class="n">memory</span><span class="p">,</span> <span class="n">serializer</span><span class="p">)</span>
<span class="n">merger</span><span class="o">.</span><span class="n">mergeCombiners</span><span class="p">(</span><span class="n">it</span><span class="p">)</span>
<span class="k">return</span> <span class="n">merger</span><span class="o">.</span><span class="n">items</span><span class="p">()</span>
<span class="k">return</span> <span class="n">shuffled</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="n">groupByKey</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span><span class="o">.</span><span class="n">mapValues</span><span class="p">(</span><span class="n">ResultIterable</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.flatMapValues"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.flatMapValues.html#pyspark.RDD.flatMapValues">[docs]</a> <span class="k">def</span> <span class="nf">flatMapValues</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Pass each value in the key-value pair RDD through a flatMap function</span>
<span class="sd"> without changing the keys; this also retains the original RDD&#39;s</span>
<span class="sd"> partitioning.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, [&quot;x&quot;, &quot;y&quot;, &quot;z&quot;]), (&quot;b&quot;, [&quot;p&quot;, &quot;r&quot;])])</span>
<span class="sd"> &gt;&gt;&gt; def f(x): return x</span>
<span class="sd"> &gt;&gt;&gt; x.flatMapValues(f).collect()</span>
<span class="sd"> [(&#39;a&#39;, &#39;x&#39;), (&#39;a&#39;, &#39;y&#39;), (&#39;a&#39;, &#39;z&#39;), (&#39;b&#39;, &#39;p&#39;), (&#39;b&#39;, &#39;r&#39;)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">flat_map_fn</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">kv</span><span class="p">:</span> <span class="p">((</span><span class="n">kv</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">x</span><span class="p">)</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">f</span><span class="p">(</span><span class="n">kv</span><span class="p">[</span><span class="mi">1</span><span class="p">]))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">flatMap</span><span class="p">(</span><span class="n">flat_map_fn</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.mapValues"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.mapValues.html#pyspark.RDD.mapValues">[docs]</a> <span class="k">def</span> <span class="nf">mapValues</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Pass each value in the key-value pair RDD through a map function</span>
<span class="sd"> without changing the keys; this also retains the original RDD&#39;s</span>
<span class="sd"> partitioning.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, [&quot;apple&quot;, &quot;banana&quot;, &quot;lemon&quot;]), (&quot;b&quot;, [&quot;grapes&quot;])])</span>
<span class="sd"> &gt;&gt;&gt; def f(x): return len(x)</span>
<span class="sd"> &gt;&gt;&gt; x.mapValues(f).collect()</span>
<span class="sd"> [(&#39;a&#39;, 3), (&#39;b&#39;, 1)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">map_values_fn</span> <span class="o">=</span> <span class="k">lambda</span> <span class="n">kv</span><span class="p">:</span> <span class="p">(</span><span class="n">kv</span><span class="p">[</span><span class="mi">0</span><span class="p">],</span> <span class="n">f</span><span class="p">(</span><span class="n">kv</span><span class="p">[</span><span class="mi">1</span><span class="p">]))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="n">map_values_fn</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.groupWith"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.groupWith.html#pyspark.RDD.groupWith">[docs]</a> <span class="k">def</span> <span class="nf">groupWith</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="o">*</span><span class="n">others</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Alias for cogroup but with support for multiple RDDs.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; w = sc.parallelize([(&quot;a&quot;, 5), (&quot;b&quot;, 6)])</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 4)])</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize([(&quot;a&quot;, 2)])</span>
<span class="sd"> &gt;&gt;&gt; z = sc.parallelize([(&quot;b&quot;, 42)])</span>
<span class="sd"> &gt;&gt;&gt; [(x, tuple(map(list, y))) for x, y in sorted(list(w.groupWith(x, y, z).collect()))]</span>
<span class="sd"> [(&#39;a&#39;, ([5], [1], [2], [])), (&#39;b&#39;, ([6], [4], [], [42]))]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">python_cogroup</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="o">+</span> <span class="n">others</span><span class="p">,</span> <span class="n">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">)</span></div>
<span class="c1"># TODO: add variant with custom partitioner</span>
<div class="viewcode-block" id="RDD.cogroup"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.cogroup.html#pyspark.RDD.cogroup">[docs]</a> <span class="k">def</span> <span class="nf">cogroup</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">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> For each key k in `self` or `other`, return a resulting RDD that</span>
<span class="sd"> contains a tuple with the list of values for that key in `self` as</span>
<span class="sd"> well as `other`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 4)])</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize([(&quot;a&quot;, 2)])</span>
<span class="sd"> &gt;&gt;&gt; [(x, tuple(map(list, y))) for x, y in sorted(list(x.cogroup(y).collect()))]</span>
<span class="sd"> [(&#39;a&#39;, ([1], [2])), (&#39;b&#39;, ([4], []))]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">python_cogroup</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">numPartitions</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.sampleByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.sampleByKey.html#pyspark.RDD.sampleByKey">[docs]</a> <span class="k">def</span> <span class="nf">sampleByKey</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">withReplacement</span><span class="p">,</span> <span class="n">fractions</span><span class="p">,</span> <span class="n">seed</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a subset of this RDD sampled by key (via stratified sampling).</span>
<span class="sd"> Create a sample of this RDD using variable sampling rates for</span>
<span class="sd"> different keys as specified by fractions, a key to sampling rate map.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; fractions = {&quot;a&quot;: 0.2, &quot;b&quot;: 0.1}</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(fractions.keys()).cartesian(sc.parallelize(range(0, 1000)))</span>
<span class="sd"> &gt;&gt;&gt; sample = dict(rdd.sampleByKey(False, fractions, 2).groupByKey().collect())</span>
<span class="sd"> &gt;&gt;&gt; 100 &lt; len(sample[&quot;a&quot;]) &lt; 300 and 50 &lt; len(sample[&quot;b&quot;]) &lt; 150</span>
<span class="sd"> True</span>
<span class="sd"> &gt;&gt;&gt; max(sample[&quot;a&quot;]) &lt;= 999 and min(sample[&quot;a&quot;]) &gt;= 0</span>
<span class="sd"> True</span>
<span class="sd"> &gt;&gt;&gt; max(sample[&quot;b&quot;]) &lt;= 999 and min(sample[&quot;b&quot;]) &gt;= 0</span>
<span class="sd"> True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">for</span> <span class="n">fraction</span> <span class="ow">in</span> <span class="n">fractions</span><span class="o">.</span><span class="n">values</span><span class="p">():</span>
<span class="k">assert</span> <span class="n">fraction</span> <span class="o">&gt;=</span> <span class="mf">0.0</span><span class="p">,</span> <span class="s2">&quot;Negative fraction value: </span><span class="si">%s</span><span class="s2">&quot;</span> <span class="o">%</span> <span class="n">fraction</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span>
<span class="n">RDDStratifiedSampler</span><span class="p">(</span><span class="n">withReplacement</span><span class="p">,</span> <span class="n">fractions</span><span class="p">,</span> <span class="n">seed</span><span class="p">)</span><span class="o">.</span><span class="n">func</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.subtractByKey"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.subtractByKey.html#pyspark.RDD.subtractByKey">[docs]</a> <span class="k">def</span> <span class="nf">subtractByKey</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">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return each (key, value) pair in `self` that has no pair with matching</span>
<span class="sd"> key in `other`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 4), (&quot;b&quot;, 5), (&quot;a&quot;, 2)])</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize([(&quot;a&quot;, 3), (&quot;c&quot;, None)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(x.subtractByKey(y).collect())</span>
<span class="sd"> [(&#39;b&#39;, 4), (&#39;b&#39;, 5)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">filter_func</span><span class="p">(</span><span class="n">pair</span><span class="p">):</span>
<span class="n">key</span><span class="p">,</span> <span class="p">(</span><span class="n">val1</span><span class="p">,</span> <span class="n">val2</span><span class="p">)</span> <span class="o">=</span> <span class="n">pair</span>
<span class="k">return</span> <span class="n">val1</span> <span class="ow">and</span> <span class="ow">not</span> <span class="n">val2</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">cogroup</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">)</span><span class="o">.</span><span class="n">filter</span><span class="p">(</span><span class="n">filter_func</span><span class="p">)</span><span class="o">.</span><span class="n">flatMapValues</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">[</span><span class="mi">0</span><span class="p">])</span></div>
<div class="viewcode-block" id="RDD.subtract"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.subtract.html#pyspark.RDD.subtract">[docs]</a> <span class="k">def</span> <span class="nf">subtract</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">numPartitions</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return each value in `self` that is not contained in `other`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize([(&quot;a&quot;, 1), (&quot;b&quot;, 4), (&quot;b&quot;, 5), (&quot;a&quot;, 3)])</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize([(&quot;a&quot;, 3), (&quot;c&quot;, None)])</span>
<span class="sd"> &gt;&gt;&gt; sorted(x.subtract(y).collect())</span>
<span class="sd"> [(&#39;a&#39;, 1), (&#39;b&#39;, 4), (&#39;b&#39;, 5)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="c1"># note: here &#39;True&#39; is just a placeholder</span>
<span class="n">rdd</span> <span class="o">=</span> <span class="n">other</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="kc">True</span><span class="p">))</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">x</span><span class="p">,</span> <span class="kc">True</span><span class="p">))</span><span class="o">.</span><span class="n">subtractByKey</span><span class="p">(</span><span class="n">rdd</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">)</span><span class="o">.</span><span class="n">keys</span><span class="p">()</span></div>
<div class="viewcode-block" id="RDD.keyBy"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.keyBy.html#pyspark.RDD.keyBy">[docs]</a> <span class="k">def</span> <span class="nf">keyBy</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Creates tuples of the elements in this RDD by applying `f`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x)</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize(zip(range(0,5), range(0,5)))</span>
<span class="sd"> &gt;&gt;&gt; [(x, list(map(list, y))) for x, y in sorted(x.cogroup(y).collect())]</span>
<span class="sd"> [(0, [[0], [0]]), (1, [[1], [1]]), (2, [[], [2]]), (3, [[], [3]]), (4, [[2], [4]])]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="p">(</span><span class="n">f</span><span class="p">(</span><span class="n">x</span><span class="p">),</span> <span class="n">x</span><span class="p">))</span></div>
<div class="viewcode-block" id="RDD.repartition"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.repartition.html#pyspark.RDD.repartition">[docs]</a> <span class="k">def</span> <span class="nf">repartition</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD that has exactly numPartitions partitions.</span>
<span class="sd"> Can increase or decrease the level of parallelism in this RDD.</span>
<span class="sd"> Internally, this uses a shuffle to redistribute data.</span>
<span class="sd"> If you are decreasing the number of partitions in this RDD, consider</span>
<span class="sd"> using `coalesce`, which can avoid performing a shuffle.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1,2,3,4,5,6,7], 4)</span>
<span class="sd"> &gt;&gt;&gt; sorted(rdd.glom().collect())</span>
<span class="sd"> [[1], [2, 3], [4, 5], [6, 7]]</span>
<span class="sd"> &gt;&gt;&gt; len(rdd.repartition(2).glom().collect())</span>
<span class="sd"> 2</span>
<span class="sd"> &gt;&gt;&gt; len(rdd.repartition(10).glom().collect())</span>
<span class="sd"> 10</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">coalesce</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">shuffle</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.coalesce"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.coalesce.html#pyspark.RDD.coalesce">[docs]</a> <span class="k">def</span> <span class="nf">coalesce</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">numPartitions</span><span class="p">,</span> <span class="n">shuffle</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return a new RDD that is reduced into `numPartitions` partitions.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3, 4, 5], 3).glom().collect()</span>
<span class="sd"> [[1], [2, 3], [4, 5]]</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([1, 2, 3, 4, 5], 3).coalesce(1).glom().collect()</span>
<span class="sd"> [[1, 2, 3, 4, 5]]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">shuffle</span><span class="p">:</span>
<span class="c1"># Decrease the batch size in order to distribute evenly the elements across output</span>
<span class="c1"># partitions. Otherwise, repartition will possibly produce highly skewed partitions.</span>
<span class="n">batchSize</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="mi">10</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_batchSize</span> <span class="ow">or</span> <span class="mi">1024</span><span class="p">)</span>
<span class="n">ser</span> <span class="o">=</span> <span class="n">BatchedSerializer</span><span class="p">(</span><span class="n">PickleSerializer</span><span class="p">(),</span> <span class="n">batchSize</span><span class="p">)</span>
<span class="n">selfCopy</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_reserialize</span><span class="p">(</span><span class="n">ser</span><span class="p">)</span>
<span class="n">jrdd_deserializer</span> <span class="o">=</span> <span class="n">selfCopy</span><span class="o">.</span><span class="n">_jrdd_deserializer</span>
<span class="n">jrdd</span> <span class="o">=</span> <span class="n">selfCopy</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">coalesce</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">shuffle</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">jrdd_deserializer</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span>
<span class="n">jrdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">coalesce</span><span class="p">(</span><span class="n">numPartitions</span><span class="p">,</span> <span class="n">shuffle</span><span class="p">)</span>
<span class="k">return</span> <span class="n">RDD</span><span class="p">(</span><span class="n">jrdd</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="p">,</span> <span class="n">jrdd_deserializer</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.zip"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.zip.html#pyspark.RDD.zip">[docs]</a> <span class="k">def</span> <span class="nf">zip</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="sd">&quot;&quot;&quot;</span>
<span class="sd"> Zips this RDD with another one, returning key-value pairs with the</span>
<span class="sd"> first element in each RDD second element in each RDD, etc. Assumes</span>
<span class="sd"> that the two RDDs have the same number of partitions and the same</span>
<span class="sd"> number of elements in each partition (e.g. one was made through</span>
<span class="sd"> a map on the other).</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; x = sc.parallelize(range(0,5))</span>
<span class="sd"> &gt;&gt;&gt; y = sc.parallelize(range(1000, 1005))</span>
<span class="sd"> &gt;&gt;&gt; x.zip(y).collect()</span>
<span class="sd"> [(0, 1000), (1, 1001), (2, 1002), (3, 1003), (4, 1004)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">get_batch_size</span><span class="p">(</span><span class="n">ser</span><span class="p">):</span>
<span class="k">if</span> <span class="nb">isinstance</span><span class="p">(</span><span class="n">ser</span><span class="p">,</span> <span class="n">BatchedSerializer</span><span class="p">):</span>
<span class="k">return</span> <span class="n">ser</span><span class="o">.</span><span class="n">batchSize</span>
<span class="k">return</span> <span class="mi">1</span> <span class="c1"># not batched</span>
<span class="k">def</span> <span class="nf">batch_as</span><span class="p">(</span><span class="n">rdd</span><span class="p">,</span> <span class="n">batchSize</span><span class="p">):</span>
<span class="k">return</span> <span class="n">rdd</span><span class="o">.</span><span class="n">_reserialize</span><span class="p">(</span><span class="n">BatchedSerializer</span><span class="p">(</span><span class="n">PickleSerializer</span><span class="p">(),</span> <span class="n">batchSize</span><span class="p">))</span>
<span class="n">my_batch</span> <span class="o">=</span> <span class="n">get_batch_size</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">)</span>
<span class="n">other_batch</span> <span class="o">=</span> <span class="n">get_batch_size</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">)</span>
<span class="k">if</span> <span class="n">my_batch</span> <span class="o">!=</span> <span class="n">other_batch</span> <span class="ow">or</span> <span class="ow">not</span> <span class="n">my_batch</span><span class="p">:</span>
<span class="c1"># use the smallest batchSize for both of them</span>
<span class="n">batchSize</span> <span class="o">=</span> <span class="nb">min</span><span class="p">(</span><span class="n">my_batch</span><span class="p">,</span> <span class="n">other_batch</span><span class="p">)</span>
<span class="k">if</span> <span class="n">batchSize</span> <span class="o">&lt;=</span> <span class="mi">0</span><span class="p">:</span>
<span class="c1"># auto batched or unlimited</span>
<span class="n">batchSize</span> <span class="o">=</span> <span class="mi">100</span>
<span class="n">other</span> <span class="o">=</span> <span class="n">batch_as</span><span class="p">(</span><span class="n">other</span><span class="p">,</span> <span class="n">batchSize</span><span class="p">)</span>
<span class="bp">self</span> <span class="o">=</span> <span class="n">batch_as</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">batchSize</span><span class="p">)</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span> <span class="o">!=</span> <span class="n">other</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">():</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;Can only zip with RDD which has the same number of partitions&quot;</span><span class="p">)</span>
<span class="c1"># There will be an Exception in JVM if there are different number</span>
<span class="c1"># of items in each partitions.</span>
<span class="n">pairRDD</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">zip</span><span class="p">(</span><span class="n">other</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">)</span>
<span class="n">deserializer</span> <span class="o">=</span> <span class="n">PairDeserializer</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">,</span>
<span class="n">other</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">)</span>
<span class="k">return</span> <span class="n">RDD</span><span class="p">(</span><span class="n">pairRDD</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="p">,</span> <span class="n">deserializer</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.zipWithIndex"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.zipWithIndex.html#pyspark.RDD.zipWithIndex">[docs]</a> <span class="k">def</span> <span class="nf">zipWithIndex</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Zips this RDD with its element indices.</span>
<span class="sd"> The ordering is first based on the partition index and then the</span>
<span class="sd"> ordering of items within each partition. So the first item in</span>
<span class="sd"> the first partition gets index 0, and the last item in the last</span>
<span class="sd"> partition receives the largest index.</span>
<span class="sd"> This method needs to trigger a spark job when this RDD contains</span>
<span class="sd"> more than one partitions.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([&quot;a&quot;, &quot;b&quot;, &quot;c&quot;, &quot;d&quot;], 3).zipWithIndex().collect()</span>
<span class="sd"> [(&#39;a&#39;, 0), (&#39;b&#39;, 1), (&#39;c&#39;, 2), (&#39;d&#39;, 3)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">starts</span> <span class="o">=</span> <span class="p">[</span><span class="mi">0</span><span class="p">]</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span> <span class="o">&gt;</span> <span class="mi">1</span><span class="p">:</span>
<span class="n">nums</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="k">lambda</span> <span class="n">it</span><span class="p">:</span> <span class="p">[</span><span class="nb">sum</span><span class="p">(</span><span class="mi">1</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">it</span><span class="p">)])</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span>
<span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="nb">range</span><span class="p">(</span><span class="nb">len</span><span class="p">(</span><span class="n">nums</span><span class="p">)</span> <span class="o">-</span> <span class="mi">1</span><span class="p">):</span>
<span class="n">starts</span><span class="o">.</span><span class="n">append</span><span class="p">(</span><span class="n">starts</span><span class="p">[</span><span class="o">-</span><span class="mi">1</span><span class="p">]</span> <span class="o">+</span> <span class="n">nums</span><span class="p">[</span><span class="n">i</span><span class="p">])</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">it</span><span class="p">):</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">it</span><span class="p">,</span> <span class="n">starts</span><span class="p">[</span><span class="n">k</span><span class="p">]):</span>
<span class="k">yield</span> <span class="n">v</span><span class="p">,</span> <span class="n">i</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">func</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.zipWithUniqueId"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.zipWithUniqueId.html#pyspark.RDD.zipWithUniqueId">[docs]</a> <span class="k">def</span> <span class="nf">zipWithUniqueId</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Zips this RDD with generated unique Long ids.</span>
<span class="sd"> Items in the kth partition will get ids k, n+k, 2*n+k, ..., where</span>
<span class="sd"> n is the number of partitions. So there may exist gaps, but this</span>
<span class="sd"> method won&#39;t trigger a spark job, which is different from</span>
<span class="sd"> :meth:`zipWithIndex`.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; sc.parallelize([&quot;a&quot;, &quot;b&quot;, &quot;c&quot;, &quot;d&quot;, &quot;e&quot;], 3).zipWithUniqueId().collect()</span>
<span class="sd"> [(&#39;a&#39;, 0), (&#39;b&#39;, 1), (&#39;c&#39;, 4), (&#39;d&#39;, 2), (&#39;e&#39;, 5)]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">n</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">k</span><span class="p">,</span> <span class="n">it</span><span class="p">):</span>
<span class="k">for</span> <span class="n">i</span><span class="p">,</span> <span class="n">v</span> <span class="ow">in</span> <span class="nb">enumerate</span><span class="p">(</span><span class="n">it</span><span class="p">):</span>
<span class="k">yield</span> <span class="n">v</span><span class="p">,</span> <span class="n">i</span> <span class="o">*</span> <span class="n">n</span> <span class="o">+</span> <span class="n">k</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitionsWithIndex</span><span class="p">(</span><span class="n">func</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.name"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.name.html#pyspark.RDD.name">[docs]</a> <span class="k">def</span> <span class="nf">name</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return the name of this RDD.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">n</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">name</span><span class="p">()</span>
<span class="k">if</span> <span class="n">n</span><span class="p">:</span>
<span class="k">return</span> <span class="n">n</span></div>
<div class="viewcode-block" id="RDD.setName"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.setName.html#pyspark.RDD.setName">[docs]</a> <span class="k">def</span> <span class="nf">setName</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"> Assign a name to this RDD.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd1 = sc.parallelize([1, 2])</span>
<span class="sd"> &gt;&gt;&gt; rdd1.setName(&#39;RDD1&#39;).name()</span>
<span class="sd"> &#39;RDD1&#39;</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">setName</span><span class="p">(</span><span class="n">name</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span></div>
<div class="viewcode-block" id="RDD.toDebugString"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.toDebugString.html#pyspark.RDD.toDebugString">[docs]</a> <span class="k">def</span> <span class="nf">toDebugString</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> A description of this RDD and its recursive dependencies for debugging.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">debug_string</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">toDebugString</span><span class="p">()</span>
<span class="k">if</span> <span class="n">debug_string</span><span class="p">:</span>
<span class="k">return</span> <span class="n">debug_string</span><span class="o">.</span><span class="n">encode</span><span class="p">(</span><span class="s1">&#39;utf-8&#39;</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.getStorageLevel"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.getStorageLevel.html#pyspark.RDD.getStorageLevel">[docs]</a> <span class="k">def</span> <span class="nf">getStorageLevel</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Get the RDD&#39;s current storage level.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd1 = sc.parallelize([1,2])</span>
<span class="sd"> &gt;&gt;&gt; rdd1.getStorageLevel()</span>
<span class="sd"> StorageLevel(False, False, False, False, 1)</span>
<span class="sd"> &gt;&gt;&gt; print(rdd1.getStorageLevel())</span>
<span class="sd"> Serialized 1x Replicated</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">java_storage_level</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">getStorageLevel</span><span class="p">()</span>
<span class="n">storage_level</span> <span class="o">=</span> <span class="n">StorageLevel</span><span class="p">(</span><span class="n">java_storage_level</span><span class="o">.</span><span class="n">useDisk</span><span class="p">(),</span>
<span class="n">java_storage_level</span><span class="o">.</span><span class="n">useMemory</span><span class="p">(),</span>
<span class="n">java_storage_level</span><span class="o">.</span><span class="n">useOffHeap</span><span class="p">(),</span>
<span class="n">java_storage_level</span><span class="o">.</span><span class="n">deserialized</span><span class="p">(),</span>
<span class="n">java_storage_level</span><span class="o">.</span><span class="n">replication</span><span class="p">())</span>
<span class="k">return</span> <span class="n">storage_level</span></div>
<span class="k">def</span> <span class="nf">_defaultReducePartitions</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns the default number of partitions to use during reduce tasks (e.g., groupBy).</span>
<span class="sd"> If spark.default.parallelism is set, then we&#39;ll use the value from SparkContext</span>
<span class="sd"> defaultParallelism, otherwise we&#39;ll use the number of partitions in this RDD.</span>
<span class="sd"> This mirrors the behavior of the Scala Partitioner#defaultPartitioner, intended to reduce</span>
<span class="sd"> the likelihood of OOMs. Once PySpark adopts Partitioner-based APIs, this behavior will</span>
<span class="sd"> be inherent.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_conf</span><span class="o">.</span><span class="n">contains</span><span class="p">(</span><span class="s2">&quot;spark.default.parallelism&quot;</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">defaultParallelism</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">getNumPartitions</span><span class="p">()</span>
<div class="viewcode-block" id="RDD.lookup"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.lookup.html#pyspark.RDD.lookup">[docs]</a> <span class="k">def</span> <span class="nf">lookup</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"> Return the list of values in the RDD for key `key`. This operation</span>
<span class="sd"> is done efficiently if the RDD has a known partitioner by only</span>
<span class="sd"> searching the partition that the key maps to.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; l = range(1000)</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(zip(l, l), 10)</span>
<span class="sd"> &gt;&gt;&gt; rdd.lookup(42) # slow</span>
<span class="sd"> [42]</span>
<span class="sd"> &gt;&gt;&gt; sorted = rdd.sortByKey()</span>
<span class="sd"> &gt;&gt;&gt; sorted.lookup(42) # fast</span>
<span class="sd"> [42]</span>
<span class="sd"> &gt;&gt;&gt; sorted.lookup(1024)</span>
<span class="sd"> []</span>
<span class="sd"> &gt;&gt;&gt; rdd2 = sc.parallelize([((&#39;a&#39;, &#39;b&#39;), &#39;c&#39;)]).groupByKey()</span>
<span class="sd"> &gt;&gt;&gt; list(rdd2.lookup((&#39;a&#39;, &#39;b&#39;))[0])</span>
<span class="sd"> [&#39;c&#39;]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">values</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">filter</span><span class="p">(</span><span class="k">lambda</span> <span class="n">kv</span><span class="p">:</span> <span class="n">kv</span><span class="p">[</span><span class="mi">0</span><span class="p">]</span> <span class="o">==</span> <span class="n">key</span><span class="p">)</span><span class="o">.</span><span class="n">values</span><span class="p">()</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">partitioner</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">runJob</span><span class="p">(</span><span class="n">values</span><span class="p">,</span> <span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">x</span><span class="p">,</span> <span class="p">[</span><span class="bp">self</span><span class="o">.</span><span class="n">partitioner</span><span class="p">(</span><span class="n">key</span><span class="p">)])</span>
<span class="k">return</span> <span class="n">values</span><span class="o">.</span><span class="n">collect</span><span class="p">()</span></div>
<span class="k">def</span> <span class="nf">_to_java_object_rdd</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot; Return a JavaRDD of Object by unpickling</span>
<span class="sd"> It will convert each Python object into Java object by Pyrolite, whenever the</span>
<span class="sd"> RDD is serialized in batch or not.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">rdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_pickled</span><span class="p">()</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">SerDeUtil</span><span class="o">.</span><span class="n">pythonToJava</span><span class="p">(</span><span class="n">rdd</span><span class="o">.</span><span class="n">_jrdd</span><span class="p">,</span> <span class="kc">True</span><span class="p">)</span>
<div class="viewcode-block" id="RDD.countApprox"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.countApprox.html#pyspark.RDD.countApprox">[docs]</a> <span class="k">def</span> <span class="nf">countApprox</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">timeout</span><span class="p">,</span> <span class="n">confidence</span><span class="o">=</span><span class="mf">0.95</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Approximate version of count() that returns a potentially incomplete</span>
<span class="sd"> result within a timeout, even if not all tasks have finished.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(range(1000), 10)</span>
<span class="sd"> &gt;&gt;&gt; rdd.countApprox(1000, 1.0)</span>
<span class="sd"> 1000</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">drdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="k">lambda</span> <span class="n">it</span><span class="p">:</span> <span class="p">[</span><span class="nb">float</span><span class="p">(</span><span class="nb">sum</span><span class="p">(</span><span class="mi">1</span> <span class="k">for</span> <span class="n">i</span> <span class="ow">in</span> <span class="n">it</span><span class="p">))])</span>
<span class="k">return</span> <span class="nb">int</span><span class="p">(</span><span class="n">drdd</span><span class="o">.</span><span class="n">sumApprox</span><span class="p">(</span><span class="n">timeout</span><span class="p">,</span> <span class="n">confidence</span><span class="p">))</span></div>
<div class="viewcode-block" id="RDD.sumApprox"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.sumApprox.html#pyspark.RDD.sumApprox">[docs]</a> <span class="k">def</span> <span class="nf">sumApprox</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">timeout</span><span class="p">,</span> <span class="n">confidence</span><span class="o">=</span><span class="mf">0.95</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Approximate operation to return the sum within a timeout</span>
<span class="sd"> or meet the confidence.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(range(1000), 10)</span>
<span class="sd"> &gt;&gt;&gt; r = sum(range(1000))</span>
<span class="sd"> &gt;&gt;&gt; abs(rdd.sumApprox(1000) - r) / r &lt; 0.05</span>
<span class="sd"> True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">jrdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">mapPartitions</span><span class="p">(</span><span class="k">lambda</span> <span class="n">it</span><span class="p">:</span> <span class="p">[</span><span class="nb">float</span><span class="p">(</span><span class="nb">sum</span><span class="p">(</span><span class="n">it</span><span class="p">))])</span><span class="o">.</span><span class="n">_to_java_object_rdd</span><span class="p">()</span>
<span class="n">jdrdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">JavaDoubleRDD</span><span class="o">.</span><span class="n">fromRDD</span><span class="p">(</span><span class="n">jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">())</span>
<span class="n">r</span> <span class="o">=</span> <span class="n">jdrdd</span><span class="o">.</span><span class="n">sumApprox</span><span class="p">(</span><span class="n">timeout</span><span class="p">,</span> <span class="n">confidence</span><span class="p">)</span><span class="o">.</span><span class="n">getFinalValue</span><span class="p">()</span>
<span class="k">return</span> <span class="n">BoundedFloat</span><span class="p">(</span><span class="n">r</span><span class="o">.</span><span class="n">mean</span><span class="p">(),</span> <span class="n">r</span><span class="o">.</span><span class="n">confidence</span><span class="p">(),</span> <span class="n">r</span><span class="o">.</span><span class="n">low</span><span class="p">(),</span> <span class="n">r</span><span class="o">.</span><span class="n">high</span><span class="p">())</span></div>
<div class="viewcode-block" id="RDD.meanApprox"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.meanApprox.html#pyspark.RDD.meanApprox">[docs]</a> <span class="k">def</span> <span class="nf">meanApprox</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">timeout</span><span class="p">,</span> <span class="n">confidence</span><span class="o">=</span><span class="mf">0.95</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Approximate operation to return the mean within a timeout</span>
<span class="sd"> or meet the confidence.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(range(1000), 10)</span>
<span class="sd"> &gt;&gt;&gt; r = sum(range(1000)) / 1000.0</span>
<span class="sd"> &gt;&gt;&gt; abs(rdd.meanApprox(1000) - r) / r &lt; 0.05</span>
<span class="sd"> True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">jrdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="nb">float</span><span class="p">)</span><span class="o">.</span><span class="n">_to_java_object_rdd</span><span class="p">()</span>
<span class="n">jdrdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">JavaDoubleRDD</span><span class="o">.</span><span class="n">fromRDD</span><span class="p">(</span><span class="n">jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">())</span>
<span class="n">r</span> <span class="o">=</span> <span class="n">jdrdd</span><span class="o">.</span><span class="n">meanApprox</span><span class="p">(</span><span class="n">timeout</span><span class="p">,</span> <span class="n">confidence</span><span class="p">)</span><span class="o">.</span><span class="n">getFinalValue</span><span class="p">()</span>
<span class="k">return</span> <span class="n">BoundedFloat</span><span class="p">(</span><span class="n">r</span><span class="o">.</span><span class="n">mean</span><span class="p">(),</span> <span class="n">r</span><span class="o">.</span><span class="n">confidence</span><span class="p">(),</span> <span class="n">r</span><span class="o">.</span><span class="n">low</span><span class="p">(),</span> <span class="n">r</span><span class="o">.</span><span class="n">high</span><span class="p">())</span></div>
<div class="viewcode-block" id="RDD.countApproxDistinct"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.countApproxDistinct.html#pyspark.RDD.countApproxDistinct">[docs]</a> <span class="k">def</span> <span class="nf">countApproxDistinct</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">relativeSD</span><span class="o">=</span><span class="mf">0.05</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return approximate number of distinct elements in the RDD.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> relativeSD : float, optional</span>
<span class="sd"> Relative accuracy. Smaller values create</span>
<span class="sd"> counters that require more space.</span>
<span class="sd"> It must be greater than 0.000017.</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> The algorithm used is based on streamlib&#39;s implementation of</span>
<span class="sd"> `&quot;HyperLogLog in Practice: Algorithmic Engineering of a State</span>
<span class="sd"> of The Art Cardinality Estimation Algorithm&quot;, available here</span>
<span class="sd"> &lt;https://doi.org/10.1145/2452376.2452456&gt;`_.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; n = sc.parallelize(range(1000)).map(str).countApproxDistinct()</span>
<span class="sd"> &gt;&gt;&gt; 900 &lt; n &lt; 1100</span>
<span class="sd"> True</span>
<span class="sd"> &gt;&gt;&gt; n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct()</span>
<span class="sd"> &gt;&gt;&gt; 16 &lt; n &lt; 24</span>
<span class="sd"> True</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">if</span> <span class="n">relativeSD</span> <span class="o">&lt;</span> <span class="mf">0.000017</span><span class="p">:</span>
<span class="k">raise</span> <span class="ne">ValueError</span><span class="p">(</span><span class="s2">&quot;relativeSD should be greater than 0.000017&quot;</span><span class="p">)</span>
<span class="c1"># the hash space in Java is 2^32</span>
<span class="n">hashRDD</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">map</span><span class="p">(</span><span class="k">lambda</span> <span class="n">x</span><span class="p">:</span> <span class="n">portable_hash</span><span class="p">(</span><span class="n">x</span><span class="p">)</span> <span class="o">&amp;</span> <span class="mh">0xFFFFFFFF</span><span class="p">)</span>
<span class="k">return</span> <span class="n">hashRDD</span><span class="o">.</span><span class="n">_to_java_object_rdd</span><span class="p">()</span><span class="o">.</span><span class="n">countApproxDistinct</span><span class="p">(</span><span class="n">relativeSD</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.toLocalIterator"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.toLocalIterator.html#pyspark.RDD.toLocalIterator">[docs]</a> <span class="k">def</span> <span class="nf">toLocalIterator</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">prefetchPartitions</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Return an iterator that contains all of the elements in this RDD.</span>
<span class="sd"> The iterator will consume as much memory as the largest partition in this RDD.</span>
<span class="sd"> With prefetch it may consume up to the memory of the 2 largest partitions.</span>
<span class="sd"> Parameters</span>
<span class="sd"> ----------</span>
<span class="sd"> prefetchPartitions : bool, optional</span>
<span class="sd"> If Spark should pre-fetch the next partition</span>
<span class="sd"> before it is needed.</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize(range(10))</span>
<span class="sd"> &gt;&gt;&gt; [x for x in rdd.toLocalIterator()]</span>
<span class="sd"> [0, 1, 2, 3, 4, 5, 6, 7, 8, 9]</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">with</span> <span class="n">SCCallSiteSync</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">context</span><span class="p">)</span> <span class="k">as</span> <span class="n">css</span><span class="p">:</span>
<span class="n">sock_info</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="o">.</span><span class="n">toLocalIteratorAndServe</span><span class="p">(</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">(),</span>
<span class="n">prefetchPartitions</span><span class="p">)</span>
<span class="k">return</span> <span class="n">_local_iterator_from_socket</span><span class="p">(</span><span class="n">sock_info</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDD.barrier"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.barrier.html#pyspark.RDD.barrier">[docs]</a> <span class="k">def</span> <span class="nf">barrier</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Marks the current stage as a barrier stage, where Spark must launch all tasks together.</span>
<span class="sd"> In case of a task failure, instead of only restarting the failed task, Spark will abort the</span>
<span class="sd"> entire stage and relaunch all tasks for this stage.</span>
<span class="sd"> The barrier execution mode feature is experimental and it only handles limited scenarios.</span>
<span class="sd"> Please read the linked SPIP and design docs to understand the limitations and future plans.</span>
<span class="sd"> .. versionadded:: 2.4.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> :class:`RDDBarrier`</span>
<span class="sd"> instance that provides actions within a barrier stage.</span>
<span class="sd"> See Also</span>
<span class="sd"> --------</span>
<span class="sd"> pyspark.BarrierTaskContext</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> For additional information see</span>
<span class="sd"> - `SPIP: Barrier Execution Mode &lt;http://jira.apache.org/jira/browse/SPARK-24374&gt;`_</span>
<span class="sd"> - `Design Doc &lt;https://jira.apache.org/jira/browse/SPARK-24582&gt;`_</span>
<span class="sd"> This API is experimental</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">RDDBarrier</span><span class="p">(</span><span class="bp">self</span><span class="p">)</span></div>
<span class="k">def</span> <span class="nf">_is_barrier</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Whether this RDD is in a barrier stage.</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">()</span><span class="o">.</span><span class="n">isBarrier</span><span class="p">()</span>
<div class="viewcode-block" id="RDD.withResources"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.withResources.html#pyspark.RDD.withResources">[docs]</a> <span class="k">def</span> <span class="nf">withResources</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">profile</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Specify a :class:`pyspark.resource.ResourceProfile` to use when calculating this RDD.</span>
<span class="sd"> This is only supported on certain cluster managers and currently requires dynamic</span>
<span class="sd"> allocation to be enabled. It will result in new executors with the resources specified</span>
<span class="sd"> being acquired to calculate the RDD.</span>
<span class="sd"> .. versionadded:: 3.1.0</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This API is experimental</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="bp">self</span><span class="o">.</span><span class="n">has_resource_profile</span> <span class="o">=</span> <span class="kc">True</span>
<span class="k">if</span> <span class="n">profile</span><span class="o">.</span><span class="n">_java_resource_profile</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="n">jrp</span> <span class="o">=</span> <span class="n">profile</span><span class="o">.</span><span class="n">_java_resource_profile</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">builder</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</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">resource</span><span class="o">.</span><span class="n">ResourceProfileBuilder</span><span class="p">()</span>
<span class="n">ereqs</span> <span class="o">=</span> <span class="n">ExecutorResourceRequests</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="p">,</span> <span class="n">profile</span><span class="o">.</span><span class="n">_executor_resource_requests</span><span class="p">)</span>
<span class="n">treqs</span> <span class="o">=</span> <span class="n">TaskResourceRequests</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="p">,</span> <span class="n">profile</span><span class="o">.</span><span class="n">_task_resource_requests</span><span class="p">)</span>
<span class="n">builder</span><span class="o">.</span><span class="n">require</span><span class="p">(</span><span class="n">ereqs</span><span class="o">.</span><span class="n">_java_executor_resource_requests</span><span class="p">)</span>
<span class="n">builder</span><span class="o">.</span><span class="n">require</span><span class="p">(</span><span class="n">treqs</span><span class="o">.</span><span class="n">_java_task_resource_requests</span><span class="p">)</span>
<span class="n">jrp</span> <span class="o">=</span> <span class="n">builder</span><span class="o">.</span><span class="n">build</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">withResources</span><span class="p">(</span><span class="n">jrp</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span></div>
<div class="viewcode-block" id="RDD.getResourceProfile"><a class="viewcode-back" href="../../reference/api/pyspark.RDD.getResourceProfile.html#pyspark.RDD.getResourceProfile">[docs]</a> <span class="k">def</span> <span class="nf">getResourceProfile</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Get the :class:`pyspark.resource.ResourceProfile` specified with this RDD or None</span>
<span class="sd"> if it wasn&#39;t specified.</span>
<span class="sd"> .. versionadded:: 3.1.0</span>
<span class="sd"> Returns</span>
<span class="sd"> -------</span>
<span class="sd"> :py:class:`pyspark.resource.ResourceProfile`</span>
<span class="sd"> The the user specified profile or None if none were specified</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This API is experimental</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="n">rp</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">getResourceProfile</span><span class="p">()</span>
<span class="k">if</span> <span class="n">rp</span> <span class="ow">is</span> <span class="ow">not</span> <span class="kc">None</span><span class="p">:</span>
<span class="k">return</span> <span class="n">ResourceProfile</span><span class="p">(</span><span class="n">_java_resource_profile</span><span class="o">=</span><span class="n">rp</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="k">return</span> <span class="kc">None</span></div></div>
<span class="k">def</span> <span class="nf">_prepare_for_python_RDD</span><span class="p">(</span><span class="n">sc</span><span class="p">,</span> <span class="n">command</span><span class="p">):</span>
<span class="c1"># the serialized command will be compressed by broadcast</span>
<span class="n">ser</span> <span class="o">=</span> <span class="n">CloudPickleSerializer</span><span class="p">()</span>
<span class="n">pickled_command</span> <span class="o">=</span> <span class="n">ser</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">command</span><span class="p">)</span>
<span class="k">if</span> <span class="nb">len</span><span class="p">(</span><span class="n">pickled_command</span><span class="p">)</span> <span class="o">&gt;</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">getBroadcastThreshold</span><span class="p">(</span><span class="n">sc</span><span class="o">.</span><span class="n">_jsc</span><span class="p">):</span> <span class="c1"># Default 1M</span>
<span class="c1"># The broadcast will have same life cycle as created PythonRDD</span>
<span class="n">broadcast</span> <span class="o">=</span> <span class="n">sc</span><span class="o">.</span><span class="n">broadcast</span><span class="p">(</span><span class="n">pickled_command</span><span class="p">)</span>
<span class="n">pickled_command</span> <span class="o">=</span> <span class="n">ser</span><span class="o">.</span><span class="n">dumps</span><span class="p">(</span><span class="n">broadcast</span><span class="p">)</span>
<span class="n">broadcast_vars</span> <span class="o">=</span> <span class="p">[</span><span class="n">x</span><span class="o">.</span><span class="n">_jbroadcast</span> <span class="k">for</span> <span class="n">x</span> <span class="ow">in</span> <span class="n">sc</span><span class="o">.</span><span class="n">_pickled_broadcast_vars</span><span class="p">]</span>
<span class="n">sc</span><span class="o">.</span><span class="n">_pickled_broadcast_vars</span><span class="o">.</span><span class="n">clear</span><span class="p">()</span>
<span class="k">return</span> <span class="n">pickled_command</span><span class="p">,</span> <span class="n">broadcast_vars</span><span class="p">,</span> <span class="n">sc</span><span class="o">.</span><span class="n">environment</span><span class="p">,</span> <span class="n">sc</span><span class="o">.</span><span class="n">_python_includes</span>
<span class="k">def</span> <span class="nf">_wrap_function</span><span class="p">(</span><span class="n">sc</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">deserializer</span><span class="p">,</span> <span class="n">serializer</span><span class="p">,</span> <span class="n">profiler</span><span class="o">=</span><span class="kc">None</span><span class="p">):</span>
<span class="k">assert</span> <span class="n">deserializer</span><span class="p">,</span> <span class="s2">&quot;deserializer should not be empty&quot;</span>
<span class="k">assert</span> <span class="n">serializer</span><span class="p">,</span> <span class="s2">&quot;serializer should not be empty&quot;</span>
<span class="n">command</span> <span class="o">=</span> <span class="p">(</span><span class="n">func</span><span class="p">,</span> <span class="n">profiler</span><span class="p">,</span> <span class="n">deserializer</span><span class="p">,</span> <span class="n">serializer</span><span class="p">)</span>
<span class="n">pickled_command</span><span class="p">,</span> <span class="n">broadcast_vars</span><span class="p">,</span> <span class="n">env</span><span class="p">,</span> <span class="n">includes</span> <span class="o">=</span> <span class="n">_prepare_for_python_RDD</span><span class="p">(</span><span class="n">sc</span><span class="p">,</span> <span class="n">command</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">PythonFunction</span><span class="p">(</span><span class="nb">bytearray</span><span class="p">(</span><span class="n">pickled_command</span><span class="p">),</span> <span class="n">env</span><span class="p">,</span> <span class="n">includes</span><span class="p">,</span> <span class="n">sc</span><span class="o">.</span><span class="n">pythonExec</span><span class="p">,</span>
<span class="n">sc</span><span class="o">.</span><span class="n">pythonVer</span><span class="p">,</span> <span class="n">broadcast_vars</span><span class="p">,</span> <span class="n">sc</span><span class="o">.</span><span class="n">_javaAccumulator</span><span class="p">)</span>
<div class="viewcode-block" id="RDDBarrier"><a class="viewcode-back" href="../../reference/api/pyspark.RDDBarrier.html#pyspark.RDDBarrier">[docs]</a><span class="k">class</span> <span class="nc">RDDBarrier</span><span class="p">(</span><span class="nb">object</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Wraps an RDD in a barrier stage, which forces Spark to launch tasks of this stage together.</span>
<span class="sd"> :class:`RDDBarrier` instances are created by :func:`RDD.barrier`.</span>
<span class="sd"> .. versionadded:: 2.4.0</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This API is experimental</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">rdd</span><span class="p">):</span>
<span class="bp">self</span><span class="o">.</span><span class="n">rdd</span> <span class="o">=</span> <span class="n">rdd</span>
<div class="viewcode-block" id="RDDBarrier.mapPartitions"><a class="viewcode-back" href="../../reference/api/pyspark.RDDBarrier.mapPartitions.html#pyspark.RDDBarrier.mapPartitions">[docs]</a> <span class="k">def</span> <span class="nf">mapPartitions</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns a new RDD by applying a function to each partition of the wrapped RDD,</span>
<span class="sd"> where tasks are launched together in a barrier stage.</span>
<span class="sd"> The interface is the same as :func:`RDD.mapPartitions`.</span>
<span class="sd"> Please see the API doc there.</span>
<span class="sd"> .. versionadded:: 2.4.0</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This API is experimental</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">def</span> <span class="nf">func</span><span class="p">(</span><span class="n">s</span><span class="p">,</span> <span class="n">iterator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">f</span><span class="p">(</span><span class="n">iterator</span><span class="p">)</span>
<span class="k">return</span> <span class="n">PipelinedRDD</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">rdd</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="p">,</span> <span class="n">isFromBarrier</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div>
<div class="viewcode-block" id="RDDBarrier.mapPartitionsWithIndex"><a class="viewcode-back" href="../../reference/api/pyspark.RDDBarrier.mapPartitionsWithIndex.html#pyspark.RDDBarrier.mapPartitionsWithIndex">[docs]</a> <span class="k">def</span> <span class="nf">mapPartitionsWithIndex</span><span class="p">(</span><span class="bp">self</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">False</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Returns a new RDD by applying a function to each partition of the wrapped RDD, while</span>
<span class="sd"> tracking the index of the original partition. And all tasks are launched together</span>
<span class="sd"> in a barrier stage.</span>
<span class="sd"> The interface is the same as :func:`RDD.mapPartitionsWithIndex`.</span>
<span class="sd"> Please see the API doc there.</span>
<span class="sd"> .. versionadded:: 3.0.0</span>
<span class="sd"> Notes</span>
<span class="sd"> -----</span>
<span class="sd"> This API is experimental</span>
<span class="sd"> &quot;&quot;&quot;</span>
<span class="k">return</span> <span class="n">PipelinedRDD</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">rdd</span><span class="p">,</span> <span class="n">f</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="p">,</span> <span class="n">isFromBarrier</span><span class="o">=</span><span class="kc">True</span><span class="p">)</span></div></div>
<span class="k">class</span> <span class="nc">PipelinedRDD</span><span class="p">(</span><span class="n">RDD</span><span class="p">):</span>
<span class="sd">&quot;&quot;&quot;</span>
<span class="sd"> Examples</span>
<span class="sd"> --------</span>
<span class="sd"> Pipelined maps:</span>
<span class="sd"> &gt;&gt;&gt; rdd = sc.parallelize([1, 2, 3, 4])</span>
<span class="sd"> &gt;&gt;&gt; rdd.map(lambda x: 2 * x).cache().map(lambda x: 2 * x).collect()</span>
<span class="sd"> [4, 8, 12, 16]</span>
<span class="sd"> &gt;&gt;&gt; rdd.map(lambda x: 2 * x).map(lambda x: 2 * x).collect()</span>
<span class="sd"> [4, 8, 12, 16]</span>
<span class="sd"> Pipelined reduces:</span>
<span class="sd"> &gt;&gt;&gt; from operator import add</span>
<span class="sd"> &gt;&gt;&gt; rdd.map(lambda x: 2 * x).reduce(add)</span>
<span class="sd"> 20</span>
<span class="sd"> &gt;&gt;&gt; rdd.flatMap(lambda x: [x, x]).reduce(add)</span>
<span class="sd"> 20</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">prev</span><span class="p">,</span> <span class="n">func</span><span class="p">,</span> <span class="n">preservesPartitioning</span><span class="o">=</span><span class="kc">False</span><span class="p">,</span> <span class="n">isFromBarrier</span><span class="o">=</span><span class="kc">False</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">prev</span><span class="p">,</span> <span class="n">PipelinedRDD</span><span class="p">)</span> <span class="ow">or</span> <span class="ow">not</span> <span class="n">prev</span><span class="o">.</span><span class="n">_is_pipelinable</span><span class="p">():</span>
<span class="c1"># This transformation is the first in its stage:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">func</span> <span class="o">=</span> <span class="n">func</span>
<span class="bp">self</span><span class="o">.</span><span class="n">preservesPartitioning</span> <span class="o">=</span> <span class="n">preservesPartitioning</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_prev_jrdd</span> <span class="o">=</span> <span class="n">prev</span><span class="o">.</span><span class="n">_jrdd</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_prev_jrdd_deserializer</span> <span class="o">=</span> <span class="n">prev</span><span class="o">.</span><span class="n">_jrdd_deserializer</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">prev_func</span> <span class="o">=</span> <span class="n">prev</span><span class="o">.</span><span class="n">func</span>
<span class="k">def</span> <span class="nf">pipeline_func</span><span class="p">(</span><span class="n">split</span><span class="p">,</span> <span class="n">iterator</span><span class="p">):</span>
<span class="k">return</span> <span class="n">func</span><span class="p">(</span><span class="n">split</span><span class="p">,</span> <span class="n">prev_func</span><span class="p">(</span><span class="n">split</span><span class="p">,</span> <span class="n">iterator</span><span class="p">))</span>
<span class="bp">self</span><span class="o">.</span><span class="n">func</span> <span class="o">=</span> <span class="n">pipeline_func</span>
<span class="bp">self</span><span class="o">.</span><span class="n">preservesPartitioning</span> <span class="o">=</span> \
<span class="n">prev</span><span class="o">.</span><span class="n">preservesPartitioning</span> <span class="ow">and</span> <span class="n">preservesPartitioning</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_prev_jrdd</span> <span class="o">=</span> <span class="n">prev</span><span class="o">.</span><span class="n">_prev_jrdd</span> <span class="c1"># maintain the pipeline</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_prev_jrdd_deserializer</span> <span class="o">=</span> <span class="n">prev</span><span class="o">.</span><span class="n">_prev_jrdd_deserializer</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_cached</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">has_resource_profile</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_checkpointed</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span> <span class="o">=</span> <span class="n">prev</span><span class="o">.</span><span class="n">ctx</span>
<span class="bp">self</span><span class="o">.</span><span class="n">prev</span> <span class="o">=</span> <span class="n">prev</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_val</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_id</span> <span class="o">=</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">serializer</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_bypass_serializer</span> <span class="o">=</span> <span class="kc">False</span>
<span class="bp">self</span><span class="o">.</span><span class="n">partitioner</span> <span class="o">=</span> <span class="n">prev</span><span class="o">.</span><span class="n">partitioner</span> <span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">preservesPartitioning</span> <span class="k">else</span> <span class="kc">None</span>
<span class="bp">self</span><span class="o">.</span><span class="n">is_barrier</span> <span class="o">=</span> <span class="n">isFromBarrier</span> <span class="ow">or</span> <span class="n">prev</span><span class="o">.</span><span class="n">_is_barrier</span><span class="p">()</span>
<span class="k">def</span> <span class="nf">getNumPartitions</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_prev_jrdd</span><span class="o">.</span><span class="n">partitions</span><span class="p">()</span><span class="o">.</span><span class="n">size</span><span class="p">()</span>
<span class="nd">@property</span>
<span class="k">def</span> <span class="nf">_jrdd</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_val</span><span class="p">:</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_val</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_bypass_serializer</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span> <span class="o">=</span> <span class="n">NoOpSerializer</span><span class="p">()</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">profiler_collector</span><span class="p">:</span>
<span class="n">profiler</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">profiler_collector</span><span class="o">.</span><span class="n">new_profiler</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="p">)</span>
<span class="k">else</span><span class="p">:</span>
<span class="n">profiler</span> <span class="o">=</span> <span class="kc">None</span>
<span class="n">wrapped_func</span> <span class="o">=</span> <span class="n">_wrap_function</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">func</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">_prev_jrdd_deserializer</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_deserializer</span><span class="p">,</span> <span class="n">profiler</span><span class="p">)</span>
<span class="n">python_rdd</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">_jvm</span><span class="o">.</span><span class="n">PythonRDD</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_prev_jrdd</span><span class="o">.</span><span class="n">rdd</span><span class="p">(),</span> <span class="n">wrapped_func</span><span class="p">,</span>
<span class="bp">self</span><span class="o">.</span><span class="n">preservesPartitioning</span><span class="p">,</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_barrier</span><span class="p">)</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_val</span> <span class="o">=</span> <span class="n">python_rdd</span><span class="o">.</span><span class="n">asJavaRDD</span><span class="p">()</span>
<span class="k">if</span> <span class="n">profiler</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_val</span><span class="o">.</span><span class="n">id</span><span class="p">()</span>
<span class="bp">self</span><span class="o">.</span><span class="n">ctx</span><span class="o">.</span><span class="n">profiler_collector</span><span class="o">.</span><span class="n">add_profiler</span><span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">_id</span><span class="p">,</span> <span class="n">profiler</span><span class="p">)</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd_val</span>
<span class="k">def</span> <span class="nf">id</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">if</span> <span class="bp">self</span><span class="o">.</span><span class="n">_id</span> <span class="ow">is</span> <span class="kc">None</span><span class="p">:</span>
<span class="bp">self</span><span class="o">.</span><span class="n">_id</span> <span class="o">=</span> <span class="bp">self</span><span class="o">.</span><span class="n">_jrdd</span><span class="o">.</span><span class="n">id</span><span class="p">()</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">_id</span>
<span class="k">def</span> <span class="nf">_is_pipelinable</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="ow">not</span> <span class="p">(</span><span class="bp">self</span><span class="o">.</span><span class="n">is_cached</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_checkpointed</span> <span class="ow">or</span> <span class="bp">self</span><span class="o">.</span><span class="n">has_resource_profile</span><span class="p">)</span>
<span class="k">def</span> <span class="nf">_is_barrier</span><span class="p">(</span><span class="bp">self</span><span class="p">):</span>
<span class="k">return</span> <span class="bp">self</span><span class="o">.</span><span class="n">is_barrier</span>
<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.context</span> <span class="kn">import</span> <span class="n">SparkContext</span>
<span class="n">globs</span> <span class="o">=</span> <span class="nb">globals</span><span class="p">()</span><span class="o">.</span><span class="n">copy</span><span class="p">()</span>
<span class="c1"># The small batch size here ensures that we see multiple batches,</span>
<span class="c1"># even in these small test examples:</span>
<span class="n">globs</span><span class="p">[</span><span class="s1">&#39;sc&#39;</span><span class="p">]</span> <span class="o">=</span> <span class="n">SparkContext</span><span class="p">(</span><span class="s1">&#39;local[4]&#39;</span><span class="p">,</span> <span class="s1">&#39;PythonTest&#39;</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">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="p">)</span>
<span class="n">globs</span><span class="p">[</span><span class="s1">&#39;sc&#39;</span><span class="p">]</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>