blob: 60ae673e30191c7380bda66808fb239d9cf29e3b [file] [log] [blame]
<!DOCTYPE html>
<!--[if lt IE 7]> <html class="no-js lt-ie9 lt-ie8 lt-ie7"> <![endif]-->
<!--[if IE 7]> <html class="no-js lt-ie9 lt-ie8"> <![endif]-->
<!--[if IE 8]> <html class="no-js lt-ie9"> <![endif]-->
<!--[if gt IE 8]><!--> <html class="no-js"> <!--<![endif]-->
<head>
<meta charset="utf-8">
<meta http-equiv="X-UA-Compatible" content="IE=edge,chrome=1">
<title>SparkR (R on Spark) - Spark 3.2.0 Documentation</title>
<link rel="stylesheet" href="css/bootstrap.min.css">
<style>
body {
padding-top: 60px;
padding-bottom: 40px;
}
</style>
<meta name="viewport" content="width=device-width">
<link rel="stylesheet" href="css/main.css">
<script src="js/vendor/modernizr-2.6.1-respond-1.1.0.min.js"></script>
<link rel="stylesheet" href="css/pygments-default.css">
<link rel="stylesheet" href="https://cdn.jsdelivr.net/npm/docsearch.js@2/dist/cdn/docsearch.min.css" />
<link rel="stylesheet" href="css/docsearch.css">
<!-- Google analytics script -->
<script type="text/javascript">
var _gaq = _gaq || [];
_gaq.push(['_setAccount', 'UA-32518208-2']);
_gaq.push(['_trackPageview']);
(function() {
var ga = document.createElement('script'); ga.type = 'text/javascript'; ga.async = true;
ga.src = ('https:' == document.location.protocol ? 'https://ssl' : 'http://www') + '.google-analytics.com/ga.js';
var s = document.getElementsByTagName('script')[0]; s.parentNode.insertBefore(ga, s);
})();
</script>
</head>
<body>
<!--[if lt IE 7]>
<p class="chromeframe">You are using an outdated browser. <a href="https://browsehappy.com/">Upgrade your browser today</a> or <a href="http://www.google.com/chromeframe/?redirect=true">install Google Chrome Frame</a> to better experience this site.</p>
<![endif]-->
<!-- This code is taken from http://twitter.github.com/bootstrap/examples/hero.html -->
<nav class="navbar fixed-top navbar-expand-md navbar-light bg-light" id="topbar">
<div class="container">
<div class="navbar-header">
<div class="navbar-brand"><a href="index.html">
<img src="img/spark-logo-hd.png" style="height:50px;"/></a><span class="version">3.2.0</span>
</div>
</div>
<button class="navbar-toggler" type="button" data-toggle="collapse"
data-target="#navbarCollapse" aria-controls="navbarCollapse"
aria-expanded="false" aria-label="Toggle navigation">
<span class="navbar-toggler-icon"></span>
</button>
<div class="collapse navbar-collapse" id="navbarCollapse">
<ul class="navbar-nav">
<!--TODO(andyk): Add class="active" attribute to li some how.-->
<li class="nav-item"><a href="index.html" class="nav-link">Overview</a></li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarQuickStart" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Programming Guides</a>
<div class="dropdown-menu" aria-labelledby="navbarQuickStart">
<a class="dropdown-item" href="quick-start.html">Quick Start</a>
<a class="dropdown-item" href="rdd-programming-guide.html">RDDs, Accumulators, Broadcasts Vars</a>
<a class="dropdown-item" href="sql-programming-guide.html">SQL, DataFrames, and Datasets</a>
<a class="dropdown-item" href="structured-streaming-programming-guide.html">Structured Streaming</a>
<a class="dropdown-item" href="streaming-programming-guide.html">Spark Streaming (DStreams)</a>
<a class="dropdown-item" href="ml-guide.html">MLlib (Machine Learning)</a>
<a class="dropdown-item" href="graphx-programming-guide.html">GraphX (Graph Processing)</a>
<a class="dropdown-item" href="sparkr.html">SparkR (R on Spark)</a>
<a class="dropdown-item" href="api/python/getting_started/index.html">PySpark (Python on Spark)</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarAPIDocs" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">API Docs</a>
<div class="dropdown-menu" aria-labelledby="navbarAPIDocs">
<a class="dropdown-item" href="api/scala/org/apache/spark/index.html">Scala</a>
<a class="dropdown-item" href="api/java/index.html">Java</a>
<a class="dropdown-item" href="api/python/index.html">Python</a>
<a class="dropdown-item" href="api/R/index.html">R</a>
<a class="dropdown-item" href="api/sql/index.html">SQL, Built-in Functions</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarDeploying" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">Deploying</a>
<div class="dropdown-menu" aria-labelledby="navbarDeploying">
<a class="dropdown-item" href="cluster-overview.html">Overview</a>
<a class="dropdown-item" href="submitting-applications.html">Submitting Applications</a>
<div class="dropdown-divider"></div>
<a class="dropdown-item" href="spark-standalone.html">Spark Standalone</a>
<a class="dropdown-item" href="running-on-mesos.html">Mesos</a>
<a class="dropdown-item" href="running-on-yarn.html">YARN</a>
<a class="dropdown-item" href="running-on-kubernetes.html">Kubernetes</a>
</div>
</li>
<li class="nav-item dropdown">
<a href="#" class="nav-link dropdown-toggle" id="navbarMore" role="button" data-toggle="dropdown" aria-haspopup="true" aria-expanded="false">More</a>
<div class="dropdown-menu" aria-labelledby="navbarMore">
<a class="dropdown-item" href="configuration.html">Configuration</a>
<a class="dropdown-item" href="monitoring.html">Monitoring</a>
<a class="dropdown-item" href="tuning.html">Tuning Guide</a>
<a class="dropdown-item" href="job-scheduling.html">Job Scheduling</a>
<a class="dropdown-item" href="security.html">Security</a>
<a class="dropdown-item" href="hardware-provisioning.html">Hardware Provisioning</a>
<a class="dropdown-item" href="migration-guide.html">Migration Guide</a>
<div class="dropdown-divider"></div>
<a class="dropdown-item" href="building-spark.html">Building Spark</a>
<a class="dropdown-item" href="https://spark.apache.org/contributing.html">Contributing to Spark</a>
<a class="dropdown-item" href="https://spark.apache.org/third-party-projects.html">Third Party Projects</a>
</div>
</li>
<li class="nav-item">
<input type="text" id="docsearch-input" placeholder="Search the docs…">
</li>
</ul>
<!--<span class="navbar-text navbar-right"><span class="version-text">v3.2.0</span></span>-->
</div>
</div>
</nav>
<div class="container-wrapper">
<div class="content mr-3" id="content">
<h1 class="title">SparkR (R on Spark)</h1>
<ul id="markdown-toc">
<li><a href="#overview" id="markdown-toc-overview">Overview</a></li>
<li><a href="#sparkdataframe" id="markdown-toc-sparkdataframe">SparkDataFrame</a> <ul>
<li><a href="#starting-up-sparksession" id="markdown-toc-starting-up-sparksession">Starting Up: SparkSession</a></li>
<li><a href="#starting-up-from-rstudio" id="markdown-toc-starting-up-from-rstudio">Starting Up from RStudio</a></li>
<li><a href="#creating-sparkdataframes" id="markdown-toc-creating-sparkdataframes">Creating SparkDataFrames</a> <ul>
<li><a href="#from-local-data-frames" id="markdown-toc-from-local-data-frames">From local data frames</a></li>
<li><a href="#from-data-sources" id="markdown-toc-from-data-sources">From Data Sources</a></li>
<li><a href="#from-hive-tables" id="markdown-toc-from-hive-tables">From Hive tables</a></li>
</ul>
</li>
<li><a href="#sparkdataframe-operations" id="markdown-toc-sparkdataframe-operations">SparkDataFrame Operations</a> <ul>
<li><a href="#selecting-rows-columns" id="markdown-toc-selecting-rows-columns">Selecting rows, columns</a></li>
<li><a href="#grouping-aggregation" id="markdown-toc-grouping-aggregation">Grouping, Aggregation</a></li>
<li><a href="#operating-on-columns" id="markdown-toc-operating-on-columns">Operating on Columns</a></li>
<li><a href="#applying-user-defined-function" id="markdown-toc-applying-user-defined-function">Applying User-Defined Function</a> <ul>
<li><a href="#run-a-given-function-on-a-large-dataset-using-dapply-or-dapplycollect" id="markdown-toc-run-a-given-function-on-a-large-dataset-using-dapply-or-dapplycollect">Run a given function on a large dataset using <code class="language-plaintext highlighter-rouge">dapply</code> or <code class="language-plaintext highlighter-rouge">dapplyCollect</code></a> <ul>
<li><a href="#dapply" id="markdown-toc-dapply">dapply</a></li>
<li><a href="#dapplycollect" id="markdown-toc-dapplycollect">dapplyCollect</a></li>
</ul>
</li>
<li><a href="#run-a-given-function-on-a-large-dataset-grouping-by-input-columns-and-using-gapply-or-gapplycollect" id="markdown-toc-run-a-given-function-on-a-large-dataset-grouping-by-input-columns-and-using-gapply-or-gapplycollect">Run a given function on a large dataset grouping by input column(s) and using <code class="language-plaintext highlighter-rouge">gapply</code> or <code class="language-plaintext highlighter-rouge">gapplyCollect</code></a> <ul>
<li><a href="#gapply" id="markdown-toc-gapply">gapply</a></li>
<li><a href="#gapplycollect" id="markdown-toc-gapplycollect">gapplyCollect</a></li>
</ul>
</li>
<li><a href="#run-local-r-functions-distributed-using-sparklapply" id="markdown-toc-run-local-r-functions-distributed-using-sparklapply">Run local R functions distributed using <code class="language-plaintext highlighter-rouge">spark.lapply</code></a> <ul>
<li><a href="#sparklapply" id="markdown-toc-sparklapply">spark.lapply</a></li>
</ul>
</li>
</ul>
</li>
<li><a href="#eager-execution" id="markdown-toc-eager-execution">Eager execution</a></li>
</ul>
</li>
<li><a href="#running-sql-queries-from-sparkr" id="markdown-toc-running-sql-queries-from-sparkr">Running SQL Queries from SparkR</a></li>
</ul>
</li>
<li><a href="#machine-learning" id="markdown-toc-machine-learning">Machine Learning</a> <ul>
<li><a href="#algorithms" id="markdown-toc-algorithms">Algorithms</a> <ul>
<li><a href="#classification" id="markdown-toc-classification">Classification</a></li>
<li><a href="#regression" id="markdown-toc-regression">Regression</a></li>
<li><a href="#tree" id="markdown-toc-tree">Tree</a></li>
<li><a href="#clustering" id="markdown-toc-clustering">Clustering</a></li>
<li><a href="#collaborative-filtering" id="markdown-toc-collaborative-filtering">Collaborative Filtering</a></li>
<li><a href="#frequent-pattern-mining" id="markdown-toc-frequent-pattern-mining">Frequent Pattern Mining</a></li>
<li><a href="#statistics" id="markdown-toc-statistics">Statistics</a></li>
</ul>
</li>
<li><a href="#model-persistence" id="markdown-toc-model-persistence">Model persistence</a></li>
</ul>
</li>
<li><a href="#data-type-mapping-between-r-and-spark" id="markdown-toc-data-type-mapping-between-r-and-spark">Data type mapping between R and Spark</a></li>
<li><a href="#structured-streaming" id="markdown-toc-structured-streaming">Structured Streaming</a></li>
<li><a href="#apache-arrow-in-sparkr" id="markdown-toc-apache-arrow-in-sparkr">Apache Arrow in SparkR</a> <ul>
<li><a href="#ensure-arrow-installed" id="markdown-toc-ensure-arrow-installed">Ensure Arrow Installed</a></li>
<li><a href="#enabling-for-conversion-tofrom-r-dataframe-dapply-and-gapply" id="markdown-toc-enabling-for-conversion-tofrom-r-dataframe-dapply-and-gapply">Enabling for Conversion to/from R DataFrame, <code class="language-plaintext highlighter-rouge">dapply</code> and <code class="language-plaintext highlighter-rouge">gapply</code></a></li>
<li><a href="#supported-sql-types" id="markdown-toc-supported-sql-types">Supported SQL Types</a></li>
</ul>
</li>
<li><a href="#r-function-name-conflicts" id="markdown-toc-r-function-name-conflicts">R Function Name Conflicts</a></li>
<li><a href="#migration-guide" id="markdown-toc-migration-guide">Migration Guide</a></li>
</ul>
<h1 id="overview">Overview</h1>
<p>SparkR is an R package that provides a light-weight frontend to use Apache Spark from R.
In Spark 3.2.0, SparkR provides a distributed data frame implementation that
supports operations like selection, filtering, aggregation etc. (similar to R data frames,
<a href="https://github.com/hadley/dplyr">dplyr</a>) but on large datasets. SparkR also supports distributed
machine learning using MLlib.</p>
<h1 id="sparkdataframe">SparkDataFrame</h1>
<p>A SparkDataFrame is a distributed collection of data organized into named columns. It is conceptually
equivalent to a table in a relational database or a data frame in R, but with richer
optimizations under the hood. SparkDataFrames can be constructed from a wide array of sources such as:
structured data files, tables in Hive, external databases, or existing local R data frames.</p>
<p>All of the examples on this page use sample data included in R or the Spark distribution and can be run using the <code class="language-plaintext highlighter-rouge">./bin/sparkR</code> shell.</p>
<h2 id="starting-up-sparksession">Starting Up: SparkSession</h2>
<div data-lang="r">
<p>The entry point into SparkR is the <code class="language-plaintext highlighter-rouge">SparkSession</code> which connects your R program to a Spark cluster.
You can create a <code class="language-plaintext highlighter-rouge">SparkSession</code> using <code class="language-plaintext highlighter-rouge">sparkR.session</code> and pass in options such as the application name, any spark packages depended on, etc. Further, you can also work with SparkDataFrames via <code class="language-plaintext highlighter-rouge">SparkSession</code>. If you are working from the <code class="language-plaintext highlighter-rouge">sparkR</code> shell, the <code class="language-plaintext highlighter-rouge">SparkSession</code> should already be created for you, and you would not need to call <code class="language-plaintext highlighter-rouge">sparkR.session</code>.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">sparkR.session</span><span class="p">()</span></code></pre></figure>
</div>
<h2 id="starting-up-from-rstudio">Starting Up from RStudio</h2>
<p>You can also start SparkR from RStudio. You can connect your R program to a Spark cluster from
RStudio, R shell, Rscript or other R IDEs. To start, make sure SPARK_HOME is set in environment
(you can check <a href="https://stat.ethz.ch/R-manual/R-devel/library/base/html/Sys.getenv.html">Sys.getenv</a>),
load the SparkR package, and call <code class="language-plaintext highlighter-rouge">sparkR.session</code> as below. It will check for the Spark installation, and, if not found, it will be downloaded and cached automatically. Alternatively, you can also run <code class="language-plaintext highlighter-rouge">install.spark</code> manually.</p>
<p>In addition to calling <code class="language-plaintext highlighter-rouge">sparkR.session</code>,
you could also specify certain Spark driver properties. Normally these
<a href="configuration.html#application-properties">Application properties</a> and
<a href="configuration.html#runtime-environment">Runtime Environment</a> cannot be set programmatically, as the
driver JVM process would have been started, in this case SparkR takes care of this for you. To set
them, pass them as you would other configuration properties in the <code class="language-plaintext highlighter-rouge">sparkConfig</code> argument to
<code class="language-plaintext highlighter-rouge">sparkR.session()</code>.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="k">if</span><span class="w"> </span><span class="p">(</span><span class="n">nchar</span><span class="p">(</span><span class="n">Sys.getenv</span><span class="p">(</span><span class="s2">"SPARK_HOME"</span><span class="p">))</span><span class="w"> </span><span class="o">&lt;</span><span class="w"> </span><span class="m">1</span><span class="p">)</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="n">Sys.setenv</span><span class="p">(</span><span class="n">SPARK_HOME</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"/home/spark"</span><span class="p">)</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="n">library</span><span class="p">(</span><span class="n">SparkR</span><span class="p">,</span><span class="w"> </span><span class="n">lib.loc</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="nf">c</span><span class="p">(</span><span class="n">file.path</span><span class="p">(</span><span class="n">Sys.getenv</span><span class="p">(</span><span class="s2">"SPARK_HOME"</span><span class="p">),</span><span class="w"> </span><span class="s2">"R"</span><span class="p">,</span><span class="w"> </span><span class="s2">"lib"</span><span class="p">)))</span><span class="w">
</span><span class="n">sparkR.session</span><span class="p">(</span><span class="n">master</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"local[*]"</span><span class="p">,</span><span class="w"> </span><span class="n">sparkConfig</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="nf">list</span><span class="p">(</span><span class="n">spark.driver.memory</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"2g"</span><span class="p">))</span></code></pre></figure>
</div>
<p>The following Spark driver properties can be set in <code class="language-plaintext highlighter-rouge">sparkConfig</code> with <code class="language-plaintext highlighter-rouge">sparkR.session</code> from RStudio:</p>
<table class="table">
<tr><th>Property Name</th><th>Property group</th><th><code>spark-submit</code> equivalent</th></tr>
<tr>
<td><code>spark.master</code></td>
<td>Application Properties</td>
<td><code>--master</code></td>
</tr>
<tr>
<td><code>spark.kerberos.keytab</code></td>
<td>Application Properties</td>
<td><code>--keytab</code></td>
</tr>
<tr>
<td><code>spark.kerberos.principal</code></td>
<td>Application Properties</td>
<td><code>--principal</code></td>
</tr>
<tr>
<td><code>spark.driver.memory</code></td>
<td>Application Properties</td>
<td><code>--driver-memory</code></td>
</tr>
<tr>
<td><code>spark.driver.extraClassPath</code></td>
<td>Runtime Environment</td>
<td><code>--driver-class-path</code></td>
</tr>
<tr>
<td><code>spark.driver.extraJavaOptions</code></td>
<td>Runtime Environment</td>
<td><code>--driver-java-options</code></td>
</tr>
<tr>
<td><code>spark.driver.extraLibraryPath</code></td>
<td>Runtime Environment</td>
<td><code>--driver-library-path</code></td>
</tr>
</table>
</div>
<h2 id="creating-sparkdataframes">Creating SparkDataFrames</h2>
<p>With a <code class="language-plaintext highlighter-rouge">SparkSession</code>, applications can create <code class="language-plaintext highlighter-rouge">SparkDataFrame</code>s from a local R data frame, from a <a href="sql-data-sources-hive-tables.html">Hive table</a>, or from other <a href="sql-data-sources.html">data sources</a>.</p>
<h3 id="from-local-data-frames">From local data frames</h3>
<p>The simplest way to create a data frame is to convert a local R data frame into a SparkDataFrame. Specifically, we can use <code class="language-plaintext highlighter-rouge">as.DataFrame</code> or <code class="language-plaintext highlighter-rouge">createDataFrame</code> and pass in the local R data frame to create a SparkDataFrame. As an example, the following creates a <code class="language-plaintext highlighter-rouge">SparkDataFrame</code> based using the <code class="language-plaintext highlighter-rouge">faithful</code> dataset from R.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">df</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">as.DataFrame</span><span class="p">(</span><span class="n">faithful</span><span class="p">)</span><span class="w">
</span><span class="c1"># Displays the first part of the SparkDataFrame</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">df</span><span class="p">)</span><span class="w">
</span><span class="c1">## eruptions waiting</span><span class="w">
</span><span class="c1">##1 3.600 79</span><span class="w">
</span><span class="c1">##2 1.800 54</span><span class="w">
</span><span class="c1">##3 3.333 74</span></code></pre></figure>
</div>
<h3 id="from-data-sources">From Data Sources</h3>
<p>SparkR supports operating on a variety of data sources through the <code class="language-plaintext highlighter-rouge">SparkDataFrame</code> interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more <a href="sql-data-sources-load-save-functions.html#manually-specifying-options">specific options</a> that are available for the built-in data sources.</p>
<p>The general method for creating SparkDataFrames from data sources is <code class="language-plaintext highlighter-rouge">read.df</code>. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically.
SparkR supports reading JSON, CSV and Parquet files natively, and through packages available from sources like <a href="https://spark.apache.org/third-party-projects.html">Third Party Projects</a>, you can find data source connectors for popular file formats like Avro. These packages can either be added by
specifying <code class="language-plaintext highlighter-rouge">--packages</code> with <code class="language-plaintext highlighter-rouge">spark-submit</code> or <code class="language-plaintext highlighter-rouge">sparkR</code> commands, or if initializing SparkSession with <code class="language-plaintext highlighter-rouge">sparkPackages</code> parameter when in an interactive R shell or from RStudio.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">sparkR.session</span><span class="p">(</span><span class="n">sparkPackages</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"org.apache.spark:spark-avro_2.12:3.2.0"</span><span class="p">)</span></code></pre></figure>
</div>
<p>We can see how to use data sources using an example JSON input file. Note that the file that is used here is <em>not</em> a typical JSON file. Each line in the file must contain a separate, self-contained valid JSON object. For more information, please see <a href="http://jsonlines.org/">JSON Lines text format, also called newline-delimited JSON</a>. As a consequence, a regular multi-line JSON file will most often fail.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">people</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">read.df</span><span class="p">(</span><span class="s2">"./examples/src/main/resources/people.json"</span><span class="p">,</span><span class="w"> </span><span class="s2">"json"</span><span class="p">)</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">people</span><span class="p">)</span><span class="w">
</span><span class="c1">## age name</span><span class="w">
</span><span class="c1">##1 NA Michael</span><span class="w">
</span><span class="c1">##2 30 Andy</span><span class="w">
</span><span class="c1">##3 19 Justin</span><span class="w">
</span><span class="c1"># SparkR automatically infers the schema from the JSON file</span><span class="w">
</span><span class="n">printSchema</span><span class="p">(</span><span class="n">people</span><span class="p">)</span><span class="w">
</span><span class="c1"># root</span><span class="w">
</span><span class="c1"># |-- age: long (nullable = true)</span><span class="w">
</span><span class="c1"># |-- name: string (nullable = true)</span><span class="w">
</span><span class="c1"># Similarly, multiple files can be read with read.json</span><span class="w">
</span><span class="n">people</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">read.json</span><span class="p">(</span><span class="nf">c</span><span class="p">(</span><span class="s2">"./examples/src/main/resources/people.json"</span><span class="p">,</span><span class="w"> </span><span class="s2">"./examples/src/main/resources/people2.json"</span><span class="p">))</span></code></pre></figure>
</div>
<p>The data sources API natively supports CSV formatted input files. For more information please refer to SparkR <a href="api/R/read.df.html">read.df</a> API documentation.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">df</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">read.df</span><span class="p">(</span><span class="n">csvPath</span><span class="p">,</span><span class="w"> </span><span class="s2">"csv"</span><span class="p">,</span><span class="w"> </span><span class="n">header</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"true"</span><span class="p">,</span><span class="w"> </span><span class="n">inferSchema</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"true"</span><span class="p">,</span><span class="w"> </span><span class="n">na.strings</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"NA"</span><span class="p">)</span></code></pre></figure>
</div>
<p>The data sources API can also be used to save out SparkDataFrames into multiple file formats. For example, we can save the SparkDataFrame from the previous example
to a Parquet file using <code class="language-plaintext highlighter-rouge">write.df</code>.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">write.df</span><span class="p">(</span><span class="n">people</span><span class="p">,</span><span class="w"> </span><span class="n">path</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"people.parquet"</span><span class="p">,</span><span class="w"> </span><span class="n">source</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"parquet"</span><span class="p">,</span><span class="w"> </span><span class="n">mode</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"overwrite"</span><span class="p">)</span></code></pre></figure>
</div>
<h3 id="from-hive-tables">From Hive tables</h3>
<p>You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with <a href="building-spark.html#building-with-hive-and-jdbc-support">Hive support</a> and more details can be found in the <a href="sql-getting-started.html#starting-point-sparksession">SQL programming guide</a>. In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (<code class="language-plaintext highlighter-rouge">enableHiveSupport = TRUE</code>).</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">sparkR.session</span><span class="p">()</span><span class="w">
</span><span class="n">sql</span><span class="p">(</span><span class="s2">"CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"</span><span class="p">)</span><span class="w">
</span><span class="n">sql</span><span class="p">(</span><span class="s2">"LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"</span><span class="p">)</span><span class="w">
</span><span class="c1"># Queries can be expressed in HiveQL.</span><span class="w">
</span><span class="n">results</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">sql</span><span class="p">(</span><span class="s2">"FROM src SELECT key, value"</span><span class="p">)</span><span class="w">
</span><span class="c1"># results is now a SparkDataFrame</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">results</span><span class="p">)</span><span class="w">
</span><span class="c1">## key value</span><span class="w">
</span><span class="c1">## 1 238 val_238</span><span class="w">
</span><span class="c1">## 2 86 val_86</span><span class="w">
</span><span class="c1">## 3 311 val_311</span></code></pre></figure>
</div>
<h2 id="sparkdataframe-operations">SparkDataFrame Operations</h2>
<p>SparkDataFrames support a number of functions to do structured data processing.
Here we include some basic examples and a complete list can be found in the <a href="api/R/index.html">API</a> docs:</p>
<h3 id="selecting-rows-columns">Selecting rows, columns</h3>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Create the SparkDataFrame</span><span class="w">
</span><span class="n">df</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">as.DataFrame</span><span class="p">(</span><span class="n">faithful</span><span class="p">)</span><span class="w">
</span><span class="c1"># Get basic information about the SparkDataFrame</span><span class="w">
</span><span class="n">df</span><span class="w">
</span><span class="c1">## SparkDataFrame[eruptions:double, waiting:double]</span><span class="w">
</span><span class="c1"># Select only the "eruptions" column</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">select</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="n">df</span><span class="o">$</span><span class="n">eruptions</span><span class="p">))</span><span class="w">
</span><span class="c1">## eruptions</span><span class="w">
</span><span class="c1">##1 3.600</span><span class="w">
</span><span class="c1">##2 1.800</span><span class="w">
</span><span class="c1">##3 3.333</span><span class="w">
</span><span class="c1"># You can also pass in column name as strings</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">select</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"eruptions"</span><span class="p">))</span><span class="w">
</span><span class="c1"># Filter the SparkDataFrame to only retain rows with wait times shorter than 50 mins</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">filter</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="n">df</span><span class="o">$</span><span class="n">waiting</span><span class="w"> </span><span class="o">&lt;</span><span class="w"> </span><span class="m">50</span><span class="p">))</span><span class="w">
</span><span class="c1">## eruptions waiting</span><span class="w">
</span><span class="c1">##1 1.750 47</span><span class="w">
</span><span class="c1">##2 1.750 47</span><span class="w">
</span><span class="c1">##3 1.867 48</span></code></pre></figure>
</div>
<h3 id="grouping-aggregation">Grouping, Aggregation</h3>
<p>SparkR data frames support a number of commonly used functions to aggregate data after grouping. For example, we can compute a histogram of the <code class="language-plaintext highlighter-rouge">waiting</code> time in the <code class="language-plaintext highlighter-rouge">faithful</code> dataset as shown below</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># We use the `n` operator to count the number of times each waiting time appears</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">summarize</span><span class="p">(</span><span class="n">groupBy</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="n">df</span><span class="o">$</span><span class="n">waiting</span><span class="p">),</span><span class="w"> </span><span class="n">count</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">n</span><span class="p">(</span><span class="n">df</span><span class="o">$</span><span class="n">waiting</span><span class="p">)))</span><span class="w">
</span><span class="c1">## waiting count</span><span class="w">
</span><span class="c1">##1 70 4</span><span class="w">
</span><span class="c1">##2 67 1</span><span class="w">
</span><span class="c1">##3 69 2</span><span class="w">
</span><span class="c1"># We can also sort the output from the aggregation to get the most common waiting times</span><span class="w">
</span><span class="n">waiting_counts</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">summarize</span><span class="p">(</span><span class="n">groupBy</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="n">df</span><span class="o">$</span><span class="n">waiting</span><span class="p">),</span><span class="w"> </span><span class="n">count</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">n</span><span class="p">(</span><span class="n">df</span><span class="o">$</span><span class="n">waiting</span><span class="p">))</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">arrange</span><span class="p">(</span><span class="n">waiting_counts</span><span class="p">,</span><span class="w"> </span><span class="n">desc</span><span class="p">(</span><span class="n">waiting_counts</span><span class="o">$</span><span class="n">count</span><span class="p">)))</span><span class="w">
</span><span class="c1">## waiting count</span><span class="w">
</span><span class="c1">##1 78 15</span><span class="w">
</span><span class="c1">##2 83 14</span><span class="w">
</span><span class="c1">##3 81 13</span></code></pre></figure>
</div>
<p>In addition to standard aggregations, SparkR supports <a href="https://en.wikipedia.org/wiki/OLAP_cube">OLAP cube</a> operators <code class="language-plaintext highlighter-rouge">cube</code>:</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">head</span><span class="p">(</span><span class="n">agg</span><span class="p">(</span><span class="n">cube</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"cyl"</span><span class="p">,</span><span class="w"> </span><span class="s2">"disp"</span><span class="p">,</span><span class="w"> </span><span class="s2">"gear"</span><span class="p">),</span><span class="w"> </span><span class="n">avg</span><span class="p">(</span><span class="n">df</span><span class="o">$</span><span class="n">mpg</span><span class="p">)))</span><span class="w">
</span><span class="c1">## cyl disp gear avg(mpg)</span><span class="w">
</span><span class="c1">##1 NA 140.8 4 22.8</span><span class="w">
</span><span class="c1">##2 4 75.7 4 30.4</span><span class="w">
</span><span class="c1">##3 8 400.0 3 19.2</span><span class="w">
</span><span class="c1">##4 8 318.0 3 15.5</span><span class="w">
</span><span class="c1">##5 NA 351.0 NA 15.8</span><span class="w">
</span><span class="c1">##6 NA 275.8 NA 16.3</span></code></pre></figure>
</div>
<p>and <code class="language-plaintext highlighter-rouge">rollup</code>:</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="n">head</span><span class="p">(</span><span class="n">agg</span><span class="p">(</span><span class="n">rollup</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="s2">"cyl"</span><span class="p">,</span><span class="w"> </span><span class="s2">"disp"</span><span class="p">,</span><span class="w"> </span><span class="s2">"gear"</span><span class="p">),</span><span class="w"> </span><span class="n">avg</span><span class="p">(</span><span class="n">df</span><span class="o">$</span><span class="n">mpg</span><span class="p">)))</span><span class="w">
</span><span class="c1">## cyl disp gear avg(mpg)</span><span class="w">
</span><span class="c1">##1 4 75.7 4 30.4</span><span class="w">
</span><span class="c1">##2 8 400.0 3 19.2</span><span class="w">
</span><span class="c1">##3 8 318.0 3 15.5</span><span class="w">
</span><span class="c1">##4 4 78.7 NA 32.4</span><span class="w">
</span><span class="c1">##5 8 304.0 3 15.2</span><span class="w">
</span><span class="c1">##6 4 79.0 NA 27.3</span></code></pre></figure>
</div>
<h3 id="operating-on-columns">Operating on Columns</h3>
<p>SparkR also provides a number of functions that can be directly applied to columns for data processing and during aggregation. The example below shows the use of basic arithmetic functions.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Convert waiting time from hours to seconds.</span><span class="w">
</span><span class="c1"># Note that we can assign this to a new column in the same SparkDataFrame</span><span class="w">
</span><span class="n">df</span><span class="o">$</span><span class="n">waiting_secs</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">df</span><span class="o">$</span><span class="n">waiting</span><span class="w"> </span><span class="o">*</span><span class="w"> </span><span class="m">60</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">df</span><span class="p">)</span><span class="w">
</span><span class="c1">## eruptions waiting waiting_secs</span><span class="w">
</span><span class="c1">##1 3.600 79 4740</span><span class="w">
</span><span class="c1">##2 1.800 54 3240</span><span class="w">
</span><span class="c1">##3 3.333 74 4440</span></code></pre></figure>
</div>
<h3 id="applying-user-defined-function">Applying User-Defined Function</h3>
<p>In SparkR, we support several kinds of User-Defined Functions:</p>
<h4 id="run-a-given-function-on-a-large-dataset-using-dapply-or-dapplycollect">Run a given function on a large dataset using <code class="language-plaintext highlighter-rouge">dapply</code> or <code class="language-plaintext highlighter-rouge">dapplyCollect</code></h4>
<h5 id="dapply">dapply</h5>
<p>Apply a function to each partition of a <code class="language-plaintext highlighter-rouge">SparkDataFrame</code>. The function to be applied to each partition of the <code class="language-plaintext highlighter-rouge">SparkDataFrame</code>
and should have only one parameter, to which a <code class="language-plaintext highlighter-rouge">data.frame</code> corresponds to each partition will be passed. The output of function should be a <code class="language-plaintext highlighter-rouge">data.frame</code>. Schema specifies the row format of the resulting a <code class="language-plaintext highlighter-rouge">SparkDataFrame</code>. It must match to <a href="#data-type-mapping-between-r-and-spark">data types</a> of returned value.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Convert waiting time from hours to seconds.</span><span class="w">
</span><span class="c1"># Note that we can apply UDF to DataFrame.</span><span class="w">
</span><span class="n">schema</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">structType</span><span class="p">(</span><span class="n">structField</span><span class="p">(</span><span class="s2">"eruptions"</span><span class="p">,</span><span class="w"> </span><span class="s2">"double"</span><span class="p">),</span><span class="w"> </span><span class="n">structField</span><span class="p">(</span><span class="s2">"waiting"</span><span class="p">,</span><span class="w"> </span><span class="s2">"double"</span><span class="p">),</span><span class="w">
</span><span class="n">structField</span><span class="p">(</span><span class="s2">"waiting_secs"</span><span class="p">,</span><span class="w"> </span><span class="s2">"double"</span><span class="p">))</span><span class="w">
</span><span class="n">df1</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">dapply</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="k">function</span><span class="p">(</span><span class="n">x</span><span class="p">)</span><span class="w"> </span><span class="p">{</span><span class="w"> </span><span class="n">x</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">cbind</span><span class="p">(</span><span class="n">x</span><span class="p">,</span><span class="w"> </span><span class="n">x</span><span class="o">$</span><span class="n">waiting</span><span class="w"> </span><span class="o">*</span><span class="w"> </span><span class="m">60</span><span class="p">)</span><span class="w"> </span><span class="p">},</span><span class="w"> </span><span class="n">schema</span><span class="p">)</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">collect</span><span class="p">(</span><span class="n">df1</span><span class="p">))</span><span class="w">
</span><span class="c1">## eruptions waiting waiting_secs</span><span class="w">
</span><span class="c1">##1 3.600 79 4740</span><span class="w">
</span><span class="c1">##2 1.800 54 3240</span><span class="w">
</span><span class="c1">##3 3.333 74 4440</span><span class="w">
</span><span class="c1">##4 2.283 62 3720</span><span class="w">
</span><span class="c1">##5 4.533 85 5100</span><span class="w">
</span><span class="c1">##6 2.883 55 3300</span></code></pre></figure>
</div>
<h5 id="dapplycollect">dapplyCollect</h5>
<p>Like <code class="language-plaintext highlighter-rouge">dapply</code>, apply a function to each partition of a <code class="language-plaintext highlighter-rouge">SparkDataFrame</code> and collect the result back. The output of function
should be a <code class="language-plaintext highlighter-rouge">data.frame</code>. But, Schema is not required to be passed. Note that <code class="language-plaintext highlighter-rouge">dapplyCollect</code> can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Convert waiting time from hours to seconds.</span><span class="w">
</span><span class="c1"># Note that we can apply UDF to DataFrame and return a R's data.frame</span><span class="w">
</span><span class="n">ldf</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">dapplyCollect</span><span class="p">(</span><span class="w">
</span><span class="n">df</span><span class="p">,</span><span class="w">
</span><span class="k">function</span><span class="p">(</span><span class="n">x</span><span class="p">)</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="n">x</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">cbind</span><span class="p">(</span><span class="n">x</span><span class="p">,</span><span class="w"> </span><span class="s2">"waiting_secs"</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">x</span><span class="o">$</span><span class="n">waiting</span><span class="w"> </span><span class="o">*</span><span class="w"> </span><span class="m">60</span><span class="p">)</span><span class="w">
</span><span class="p">})</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">ldf</span><span class="p">,</span><span class="w"> </span><span class="m">3</span><span class="p">)</span><span class="w">
</span><span class="c1">## eruptions waiting waiting_secs</span><span class="w">
</span><span class="c1">##1 3.600 79 4740</span><span class="w">
</span><span class="c1">##2 1.800 54 3240</span><span class="w">
</span><span class="c1">##3 3.333 74 4440</span></code></pre></figure>
</div>
<h4 id="run-a-given-function-on-a-large-dataset-grouping-by-input-columns-and-using-gapply-or-gapplycollect">Run a given function on a large dataset grouping by input column(s) and using <code class="language-plaintext highlighter-rouge">gapply</code> or <code class="language-plaintext highlighter-rouge">gapplyCollect</code></h4>
<h5 id="gapply">gapply</h5>
<p>Apply a function to each group of a <code class="language-plaintext highlighter-rouge">SparkDataFrame</code>. The function is to be applied to each group of the <code class="language-plaintext highlighter-rouge">SparkDataFrame</code> and should have only two parameters: grouping key and R <code class="language-plaintext highlighter-rouge">data.frame</code> corresponding to
that key. The groups are chosen from <code class="language-plaintext highlighter-rouge">SparkDataFrame</code>s column(s).
The output of function should be a <code class="language-plaintext highlighter-rouge">data.frame</code>. Schema specifies the row format of the resulting
<code class="language-plaintext highlighter-rouge">SparkDataFrame</code>. It must represent R function&#8217;s output schema on the basis of Spark <a href="#data-type-mapping-between-r-and-spark">data types</a>. The column names of the returned <code class="language-plaintext highlighter-rouge">data.frame</code> are set by user.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Determine six waiting times with the largest eruption time in minutes.</span><span class="w">
</span><span class="n">schema</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">structType</span><span class="p">(</span><span class="n">structField</span><span class="p">(</span><span class="s2">"waiting"</span><span class="p">,</span><span class="w"> </span><span class="s2">"double"</span><span class="p">),</span><span class="w"> </span><span class="n">structField</span><span class="p">(</span><span class="s2">"max_eruption"</span><span class="p">,</span><span class="w"> </span><span class="s2">"double"</span><span class="p">))</span><span class="w">
</span><span class="n">result</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">gapply</span><span class="p">(</span><span class="w">
</span><span class="n">df</span><span class="p">,</span><span class="w">
</span><span class="s2">"waiting"</span><span class="p">,</span><span class="w">
</span><span class="k">function</span><span class="p">(</span><span class="n">key</span><span class="p">,</span><span class="w"> </span><span class="n">x</span><span class="p">)</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="n">y</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">data.frame</span><span class="p">(</span><span class="n">key</span><span class="p">,</span><span class="w"> </span><span class="nf">max</span><span class="p">(</span><span class="n">x</span><span class="o">$</span><span class="n">eruptions</span><span class="p">))</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="n">schema</span><span class="p">)</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">collect</span><span class="p">(</span><span class="n">arrange</span><span class="p">(</span><span class="n">result</span><span class="p">,</span><span class="w"> </span><span class="s2">"max_eruption"</span><span class="p">,</span><span class="w"> </span><span class="n">decreasing</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="kc">TRUE</span><span class="p">)))</span><span class="w">
</span><span class="c1">## waiting max_eruption</span><span class="w">
</span><span class="c1">##1 64 5.100</span><span class="w">
</span><span class="c1">##2 69 5.067</span><span class="w">
</span><span class="c1">##3 71 5.033</span><span class="w">
</span><span class="c1">##4 87 5.000</span><span class="w">
</span><span class="c1">##5 63 4.933</span><span class="w">
</span><span class="c1">##6 89 4.900</span></code></pre></figure>
</div>
<h5 id="gapplycollect">gapplyCollect</h5>
<p>Like <code class="language-plaintext highlighter-rouge">gapply</code>, applies a function to each partition of a <code class="language-plaintext highlighter-rouge">SparkDataFrame</code> and collect the result back to R data.frame. The output of the function should be a <code class="language-plaintext highlighter-rouge">data.frame</code>. But, the schema is not required to be passed. Note that <code class="language-plaintext highlighter-rouge">gapplyCollect</code> can fail if the output of UDF run on all the partition cannot be pulled to the driver and fit in driver memory.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Determine six waiting times with the largest eruption time in minutes.</span><span class="w">
</span><span class="n">result</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">gapplyCollect</span><span class="p">(</span><span class="w">
</span><span class="n">df</span><span class="p">,</span><span class="w">
</span><span class="s2">"waiting"</span><span class="p">,</span><span class="w">
</span><span class="k">function</span><span class="p">(</span><span class="n">key</span><span class="p">,</span><span class="w"> </span><span class="n">x</span><span class="p">)</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="n">y</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">data.frame</span><span class="p">(</span><span class="n">key</span><span class="p">,</span><span class="w"> </span><span class="nf">max</span><span class="p">(</span><span class="n">x</span><span class="o">$</span><span class="n">eruptions</span><span class="p">))</span><span class="w">
</span><span class="n">colnames</span><span class="p">(</span><span class="n">y</span><span class="p">)</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="nf">c</span><span class="p">(</span><span class="s2">"waiting"</span><span class="p">,</span><span class="w"> </span><span class="s2">"max_eruption"</span><span class="p">)</span><span class="w">
</span><span class="n">y</span><span class="w">
</span><span class="p">})</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">result</span><span class="p">[</span><span class="n">order</span><span class="p">(</span><span class="n">result</span><span class="o">$</span><span class="n">max_eruption</span><span class="p">,</span><span class="w"> </span><span class="n">decreasing</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="kc">TRUE</span><span class="p">),</span><span class="w"> </span><span class="p">])</span><span class="w">
</span><span class="c1">## waiting max_eruption</span><span class="w">
</span><span class="c1">##1 64 5.100</span><span class="w">
</span><span class="c1">##2 69 5.067</span><span class="w">
</span><span class="c1">##3 71 5.033</span><span class="w">
</span><span class="c1">##4 87 5.000</span><span class="w">
</span><span class="c1">##5 63 4.933</span><span class="w">
</span><span class="c1">##6 89 4.900</span></code></pre></figure>
</div>
<h4 id="run-local-r-functions-distributed-using-sparklapply">Run local R functions distributed using <code class="language-plaintext highlighter-rouge">spark.lapply</code></h4>
<h5 id="sparklapply">spark.lapply</h5>
<p>Similar to <code class="language-plaintext highlighter-rouge">lapply</code> in native R, <code class="language-plaintext highlighter-rouge">spark.lapply</code> runs a function over a list of elements and distributes the computations with Spark.
Applies a function in a manner that is similar to <code class="language-plaintext highlighter-rouge">doParallel</code> or <code class="language-plaintext highlighter-rouge">lapply</code> to elements of a list. The results of all the computations
should fit in a single machine. If that is not the case they can do something like <code class="language-plaintext highlighter-rouge">df &lt;- createDataFrame(list)</code> and then use
<code class="language-plaintext highlighter-rouge">dapply</code></p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Perform distributed training of multiple models with spark.lapply. Here, we pass</span><span class="w">
</span><span class="c1"># a read-only list of arguments which specifies family the generalized linear model should be.</span><span class="w">
</span><span class="n">families</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="nf">c</span><span class="p">(</span><span class="s2">"gaussian"</span><span class="p">,</span><span class="w"> </span><span class="s2">"poisson"</span><span class="p">)</span><span class="w">
</span><span class="n">train</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="k">function</span><span class="p">(</span><span class="n">family</span><span class="p">)</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="n">model</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">glm</span><span class="p">(</span><span class="n">Sepal.Length</span><span class="w"> </span><span class="o">~</span><span class="w"> </span><span class="n">Sepal.Width</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="n">Species</span><span class="p">,</span><span class="w"> </span><span class="n">iris</span><span class="p">,</span><span class="w"> </span><span class="n">family</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">family</span><span class="p">)</span><span class="w">
</span><span class="n">summary</span><span class="p">(</span><span class="n">model</span><span class="p">)</span><span class="w">
</span><span class="p">}</span><span class="w">
</span><span class="c1"># Return a list of model's summaries</span><span class="w">
</span><span class="n">model.summaries</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">spark.lapply</span><span class="p">(</span><span class="n">families</span><span class="p">,</span><span class="w"> </span><span class="n">train</span><span class="p">)</span><span class="w">
</span><span class="c1"># Print the summary of each model</span><span class="w">
</span><span class="n">print</span><span class="p">(</span><span class="n">model.summaries</span><span class="p">)</span></code></pre></figure>
</div>
<h3 id="eager-execution">Eager execution</h3>
<p>If eager execution is enabled, the data will be returned to R client immediately when the <code class="language-plaintext highlighter-rouge">SparkDataFrame</code> is created. By default, eager execution is not enabled and can be enabled by setting the configuration property <code class="language-plaintext highlighter-rouge">spark.sql.repl.eagerEval.enabled</code> to <code class="language-plaintext highlighter-rouge">true</code> when the <code class="language-plaintext highlighter-rouge">SparkSession</code> is started up.</p>
<p>Maximum number of rows and maximum number of characters per column of data to display can be controlled by <code class="language-plaintext highlighter-rouge">spark.sql.repl.eagerEval.maxNumRows</code> and <code class="language-plaintext highlighter-rouge">spark.sql.repl.eagerEval.truncate</code> configuration properties, respectively. These properties are only effective when eager execution is enabled. If these properties are not set explicitly, by default, data up to 20 rows and up to 20 characters per column will be showed.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Start up spark session with eager execution enabled</span><span class="w">
</span><span class="n">sparkR.session</span><span class="p">(</span><span class="n">master</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"local[*]"</span><span class="p">,</span><span class="w">
</span><span class="n">sparkConfig</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="nf">list</span><span class="p">(</span><span class="n">spark.sql.repl.eagerEval.enabled</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"true"</span><span class="p">,</span><span class="w">
</span><span class="n">spark.sql.repl.eagerEval.maxNumRows</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="nf">as.integer</span><span class="p">(</span><span class="m">10</span><span class="p">)))</span><span class="w">
</span><span class="c1"># Create a grouped and sorted SparkDataFrame</span><span class="w">
</span><span class="n">df</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">faithful</span><span class="p">)</span><span class="w">
</span><span class="n">df2</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">arrange</span><span class="p">(</span><span class="n">summarize</span><span class="p">(</span><span class="n">groupBy</span><span class="p">(</span><span class="n">df</span><span class="p">,</span><span class="w"> </span><span class="n">df</span><span class="o">$</span><span class="n">waiting</span><span class="p">),</span><span class="w"> </span><span class="n">count</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">n</span><span class="p">(</span><span class="n">df</span><span class="o">$</span><span class="n">waiting</span><span class="p">)),</span><span class="w"> </span><span class="s2">"waiting"</span><span class="p">)</span><span class="w">
</span><span class="c1"># Similar to R data.frame, displays the data returned, instead of SparkDataFrame class string</span><span class="w">
</span><span class="n">df2</span><span class="w">
</span><span class="c1">##+-------+-----+</span><span class="w">
</span><span class="c1">##|waiting|count|</span><span class="w">
</span><span class="c1">##+-------+-----+</span><span class="w">
</span><span class="c1">##| 43.0| 1|</span><span class="w">
</span><span class="c1">##| 45.0| 3|</span><span class="w">
</span><span class="c1">##| 46.0| 5|</span><span class="w">
</span><span class="c1">##| 47.0| 4|</span><span class="w">
</span><span class="c1">##| 48.0| 3|</span><span class="w">
</span><span class="c1">##| 49.0| 5|</span><span class="w">
</span><span class="c1">##| 50.0| 5|</span><span class="w">
</span><span class="c1">##| 51.0| 6|</span><span class="w">
</span><span class="c1">##| 52.0| 5|</span><span class="w">
</span><span class="c1">##| 53.0| 7|</span><span class="w">
</span><span class="c1">##+-------+-----+</span><span class="w">
</span><span class="c1">##only showing top 10 rows</span></code></pre></figure>
</div>
<p>Note that to enable eager execution in <code class="language-plaintext highlighter-rouge">sparkR</code> shell, add <code class="language-plaintext highlighter-rouge">spark.sql.repl.eagerEval.enabled=true</code> configuration property to the <code class="language-plaintext highlighter-rouge">--conf</code> option.</p>
<h2 id="running-sql-queries-from-sparkr">Running SQL Queries from SparkR</h2>
<p>A SparkDataFrame can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data.
The <code class="language-plaintext highlighter-rouge">sql</code> function enables applications to run SQL queries programmatically and returns the result as a <code class="language-plaintext highlighter-rouge">SparkDataFrame</code>.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Load a JSON file</span><span class="w">
</span><span class="n">people</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">read.df</span><span class="p">(</span><span class="s2">"./examples/src/main/resources/people.json"</span><span class="p">,</span><span class="w"> </span><span class="s2">"json"</span><span class="p">)</span><span class="w">
</span><span class="c1"># Register this SparkDataFrame as a temporary view.</span><span class="w">
</span><span class="n">createOrReplaceTempView</span><span class="p">(</span><span class="n">people</span><span class="p">,</span><span class="w"> </span><span class="s2">"people"</span><span class="p">)</span><span class="w">
</span><span class="c1"># SQL statements can be run by using the sql method</span><span class="w">
</span><span class="n">teenagers</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">sql</span><span class="p">(</span><span class="s2">"SELECT name FROM people WHERE age &gt;= 13 AND age &lt;= 19"</span><span class="p">)</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">teenagers</span><span class="p">)</span><span class="w">
</span><span class="c1">## name</span><span class="w">
</span><span class="c1">##1 Justin</span></code></pre></figure>
</div>
<h1 id="machine-learning">Machine Learning</h1>
<h2 id="algorithms">Algorithms</h2>
<p>SparkR supports the following machine learning algorithms currently:</p>
<h4 id="classification">Classification</h4>
<ul>
<li><a href="api/R/spark.logit.html"><code class="language-plaintext highlighter-rouge">spark.logit</code></a>: <a href="ml-classification-regression.html#logistic-regression"><code class="language-plaintext highlighter-rouge">Logistic Regression</code></a></li>
<li><a href="api/R/spark.mlp.html"><code class="language-plaintext highlighter-rouge">spark.mlp</code></a>: <a href="ml-classification-regression.html#multilayer-perceptron-classifier"><code class="language-plaintext highlighter-rouge">Multilayer Perceptron (MLP)</code></a></li>
<li><a href="api/R/spark.naiveBayes.html"><code class="language-plaintext highlighter-rouge">spark.naiveBayes</code></a>: <a href="ml-classification-regression.html#naive-bayes"><code class="language-plaintext highlighter-rouge">Naive Bayes</code></a></li>
<li><a href="api/R/spark.svmLinear.html"><code class="language-plaintext highlighter-rouge">spark.svmLinear</code></a>: <a href="ml-classification-regression.html#linear-support-vector-machine"><code class="language-plaintext highlighter-rouge">Linear Support Vector Machine</code></a></li>
<li><a href="api/R/fmClassifier.html"><code class="language-plaintext highlighter-rouge">spark.fmClassifier</code></a>: <a href="ml-classification-regression.html#factorization-machines-classifier"><code class="language-plaintext highlighter-rouge">Factorization Machines classifier</code></a></li>
</ul>
<h4 id="regression">Regression</h4>
<ul>
<li><a href="api/R/spark.survreg.html"><code class="language-plaintext highlighter-rouge">spark.survreg</code></a>: <a href="ml-classification-regression.html#survival-regression"><code class="language-plaintext highlighter-rouge">Accelerated Failure Time (AFT) Survival Model</code></a></li>
<li><a href="api/R/spark.glm.html"><code class="language-plaintext highlighter-rouge">spark.glm</code></a> or <a href="api/R/glm.html"><code class="language-plaintext highlighter-rouge">glm</code></a>: <a href="ml-classification-regression.html#generalized-linear-regression"><code class="language-plaintext highlighter-rouge">Generalized Linear Model (GLM)</code></a></li>
<li><a href="api/R/spark.isoreg.html"><code class="language-plaintext highlighter-rouge">spark.isoreg</code></a>: <a href="ml-classification-regression.html#isotonic-regression"><code class="language-plaintext highlighter-rouge">Isotonic Regression</code></a></li>
<li><a href="api/R/spark.lm.html"><code class="language-plaintext highlighter-rouge">spark.lm</code></a>: <a href="ml-classification-regression.html#linear-regression"><code class="language-plaintext highlighter-rouge">Linear Regression</code></a></li>
<li><a href="api/R/spark.fmRegressor.html"><code class="language-plaintext highlighter-rouge">spark.fmRegressor</code></a>: <a href="ml-classification-regression.html#factorization-machines-regressor"><code class="language-plaintext highlighter-rouge">Factorization Machines regressor</code></a></li>
</ul>
<h4 id="tree">Tree</h4>
<ul>
<li><a href="api/R/spark.decisionTree.html"><code class="language-plaintext highlighter-rouge">spark.decisionTree</code></a>: <code class="language-plaintext highlighter-rouge">Decision Tree for</code> <a href="ml-classification-regression.html#decision-tree-regression"><code class="language-plaintext highlighter-rouge">Regression</code></a> <code class="language-plaintext highlighter-rouge">and</code> <a href="ml-classification-regression.html#decision-tree-classifier"><code class="language-plaintext highlighter-rouge">Classification</code></a></li>
<li><a href="api/R/spark.gbt.html"><code class="language-plaintext highlighter-rouge">spark.gbt</code></a>: <code class="language-plaintext highlighter-rouge">Gradient Boosted Trees for</code> <a href="ml-classification-regression.html#gradient-boosted-tree-regression"><code class="language-plaintext highlighter-rouge">Regression</code></a> <code class="language-plaintext highlighter-rouge">and</code> <a href="ml-classification-regression.html#gradient-boosted-tree-classifier"><code class="language-plaintext highlighter-rouge">Classification</code></a></li>
<li><a href="api/R/spark.randomForest.html"><code class="language-plaintext highlighter-rouge">spark.randomForest</code></a>: <code class="language-plaintext highlighter-rouge">Random Forest for</code> <a href="ml-classification-regression.html#random-forest-regression"><code class="language-plaintext highlighter-rouge">Regression</code></a> <code class="language-plaintext highlighter-rouge">and</code> <a href="ml-classification-regression.html#random-forest-classifier"><code class="language-plaintext highlighter-rouge">Classification</code></a></li>
</ul>
<h4 id="clustering">Clustering</h4>
<ul>
<li><a href="api/R/spark.bisectingKmeans.html"><code class="language-plaintext highlighter-rouge">spark.bisectingKmeans</code></a>: <a href="ml-clustering.html#bisecting-k-means"><code class="language-plaintext highlighter-rouge">Bisecting k-means</code></a></li>
<li><a href="api/R/spark.gaussianMixture.html"><code class="language-plaintext highlighter-rouge">spark.gaussianMixture</code></a>: <a href="ml-clustering.html#gaussian-mixture-model-gmm"><code class="language-plaintext highlighter-rouge">Gaussian Mixture Model (GMM)</code></a></li>
<li><a href="api/R/spark.kmeans.html"><code class="language-plaintext highlighter-rouge">spark.kmeans</code></a>: <a href="ml-clustering.html#k-means"><code class="language-plaintext highlighter-rouge">K-Means</code></a></li>
<li><a href="api/R/spark.lda.html"><code class="language-plaintext highlighter-rouge">spark.lda</code></a>: <a href="ml-clustering.html#latent-dirichlet-allocation-lda"><code class="language-plaintext highlighter-rouge">Latent Dirichlet Allocation (LDA)</code></a></li>
<li><a href="api/R/spark.powerIterationClustering.html"><code class="language-plaintext highlighter-rouge">spark.powerIterationClustering (PIC)</code></a>: <a href="ml-clustering.html#power-iteration-clustering-pic"><code class="language-plaintext highlighter-rouge">Power Iteration Clustering (PIC)</code></a></li>
</ul>
<h4 id="collaborative-filtering">Collaborative Filtering</h4>
<ul>
<li><a href="api/R/spark.als.html"><code class="language-plaintext highlighter-rouge">spark.als</code></a>: <a href="ml-collaborative-filtering.html#collaborative-filtering"><code class="language-plaintext highlighter-rouge">Alternating Least Squares (ALS)</code></a></li>
</ul>
<h4 id="frequent-pattern-mining">Frequent Pattern Mining</h4>
<ul>
<li><a href="api/R/spark.fpGrowth.html"><code class="language-plaintext highlighter-rouge">spark.fpGrowth</code></a> : <a href="ml-frequent-pattern-mining.html#fp-growth"><code class="language-plaintext highlighter-rouge">FP-growth</code></a></li>
<li><a href="api/R/spark.prefixSpan.html"><code class="language-plaintext highlighter-rouge">spark.prefixSpan</code></a> : <a href="ml-frequent-pattern-mining.html#prefixSpan"><code class="language-plaintext highlighter-rouge">PrefixSpan</code></a></li>
</ul>
<h4 id="statistics">Statistics</h4>
<ul>
<li><a href="api/R/spark.kstest.html"><code class="language-plaintext highlighter-rouge">spark.kstest</code></a>: <code class="language-plaintext highlighter-rouge">Kolmogorov-Smirnov Test</code></li>
</ul>
<p>Under the hood, SparkR uses MLlib to train the model. Please refer to the corresponding section of MLlib user guide for example code.
Users can call <code class="language-plaintext highlighter-rouge">summary</code> to print a summary of the fitted model, <a href="api/R/predict.html">predict</a> to make predictions on new data, and <a href="api/R/write.ml.html">write.ml</a>/<a href="api/R/read.ml.html">read.ml</a> to save/load fitted models.
SparkR supports a subset of the available R formula operators for model fitting, including ‘~’, ‘.’, ‘:’, ‘+’, and ‘-‘.</p>
<h2 id="model-persistence">Model persistence</h2>
<p>The following example shows how to save/load a MLlib model by SparkR.</p>
<div class="highlight"><pre class="codehilite"><code><span class="n">training</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">read.df</span><span class="p">(</span><span class="s2">"data/mllib/sample_multiclass_classification_data.txt"</span><span class="p">,</span><span class="w"> </span><span class="n">source</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"libsvm"</span><span class="p">)</span><span class="w">
</span><span class="c1"># Fit a generalized linear model of family "gaussian" with spark.glm</span><span class="w">
</span><span class="n">df_list</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">randomSplit</span><span class="p">(</span><span class="n">training</span><span class="p">,</span><span class="w"> </span><span class="nf">c</span><span class="p">(</span><span class="m">7</span><span class="p">,</span><span class="m">3</span><span class="p">),</span><span class="w"> </span><span class="m">2</span><span class="p">)</span><span class="w">
</span><span class="n">gaussianDF</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">df_list</span><span class="p">[[</span><span class="m">1</span><span class="p">]]</span><span class="w">
</span><span class="n">gaussianTestDF</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">df_list</span><span class="p">[[</span><span class="m">2</span><span class="p">]]</span><span class="w">
</span><span class="n">gaussianGLM</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">spark.glm</span><span class="p">(</span><span class="n">gaussianDF</span><span class="p">,</span><span class="w"> </span><span class="n">label</span><span class="w"> </span><span class="o">~</span><span class="w"> </span><span class="n">features</span><span class="p">,</span><span class="w"> </span><span class="n">family</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"gaussian"</span><span class="p">)</span><span class="w">
</span><span class="c1"># Save and then load a fitted MLlib model</span><span class="w">
</span><span class="n">modelPath</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">tempfile</span><span class="p">(</span><span class="n">pattern</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"ml"</span><span class="p">,</span><span class="w"> </span><span class="n">fileext</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">".tmp"</span><span class="p">)</span><span class="w">
</span><span class="n">write.ml</span><span class="p">(</span><span class="n">gaussianGLM</span><span class="p">,</span><span class="w"> </span><span class="n">modelPath</span><span class="p">)</span><span class="w">
</span><span class="n">gaussianGLM2</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">read.ml</span><span class="p">(</span><span class="n">modelPath</span><span class="p">)</span><span class="w">
</span><span class="c1"># Check model summary</span><span class="w">
</span><span class="n">summary</span><span class="p">(</span><span class="n">gaussianGLM2</span><span class="p">)</span><span class="w">
</span><span class="c1"># Check model prediction</span><span class="w">
</span><span class="n">gaussianPredictions</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">predict</span><span class="p">(</span><span class="n">gaussianGLM2</span><span class="p">,</span><span class="w"> </span><span class="n">gaussianTestDF</span><span class="p">)</span><span class="w">
</span><span class="n">head</span><span class="p">(</span><span class="n">gaussianPredictions</span><span class="p">)</span><span class="w">
</span><span class="n">unlink</span><span class="p">(</span><span class="n">modelPath</span><span class="p">)</span></code></pre></div>
<div><small>Find full example code at "examples/src/main/r/ml/ml.R" in the Spark repo.</small></div>
<h1 id="data-type-mapping-between-r-and-spark">Data type mapping between R and Spark</h1>
<table class="table">
<tr><th>R</th><th>Spark</th></tr>
<tr>
<td>byte</td>
<td>byte</td>
</tr>
<tr>
<td>integer</td>
<td>integer</td>
</tr>
<tr>
<td>float</td>
<td>float</td>
</tr>
<tr>
<td>double</td>
<td>double</td>
</tr>
<tr>
<td>numeric</td>
<td>double</td>
</tr>
<tr>
<td>character</td>
<td>string</td>
</tr>
<tr>
<td>string</td>
<td>string</td>
</tr>
<tr>
<td>binary</td>
<td>binary</td>
</tr>
<tr>
<td>raw</td>
<td>binary</td>
</tr>
<tr>
<td>logical</td>
<td>boolean</td>
</tr>
<tr>
<td><a href="https://stat.ethz.ch/R-manual/R-devel/library/base/html/DateTimeClasses.html">POSIXct</a></td>
<td>timestamp</td>
</tr>
<tr>
<td><a href="https://stat.ethz.ch/R-manual/R-devel/library/base/html/DateTimeClasses.html">POSIXlt</a></td>
<td>timestamp</td>
</tr>
<tr>
<td><a href="https://stat.ethz.ch/R-manual/R-devel/library/base/html/Dates.html">Date</a></td>
<td>date</td>
</tr>
<tr>
<td>array</td>
<td>array</td>
</tr>
<tr>
<td>list</td>
<td>array</td>
</tr>
<tr>
<td>env</td>
<td>map</td>
</tr>
</table>
<h1 id="structured-streaming">Structured Streaming</h1>
<p>SparkR supports the Structured Streaming API. Structured Streaming is a scalable and fault-tolerant stream processing engine built on the Spark SQL engine. For more information see the R API on the <a href="structured-streaming-programming-guide.html">Structured Streaming Programming Guide</a></p>
<h1 id="apache-arrow-in-sparkr">Apache Arrow in SparkR</h1>
<p>Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer data between JVM and R processes. See also PySpark optimization done, <a href="sql-pyspark-pandas-with-arrow.html">PySpark Usage Guide for Pandas with Apache Arrow</a>. This guide targets to explain how to use Arrow optimization in SparkR with some key points.</p>
<h2 id="ensure-arrow-installed">Ensure Arrow Installed</h2>
<p>Arrow R library is available on CRAN and it can be installed as below.</p>
<div class="language-bash highlighter-rouge"><div class="highlight"><pre class="highlight"><code>Rscript <span class="nt">-e</span> <span class="s1">'install.packages("arrow", repos="https://cloud.r-project.org/")'</span>
</code></pre></div></div>
<p>Please refer <a href="https://arrow.apache.org/docs/r/">the official documentation of Apache Arrow</a> for more details.</p>
<p>Note that you must ensure that Arrow R package is installed and available on all cluster nodes.
The current supported minimum version is 1.0.0; however, this might change between the minor releases since Arrow optimization in SparkR is experimental.</p>
<h2 id="enabling-for-conversion-tofrom-r-dataframe-dapply-and-gapply">Enabling for Conversion to/from R DataFrame, <code class="language-plaintext highlighter-rouge">dapply</code> and <code class="language-plaintext highlighter-rouge">gapply</code></h2>
<p>Arrow optimization is available when converting a Spark DataFrame to an R DataFrame using the call <code class="language-plaintext highlighter-rouge">collect(spark_df)</code>,
when creating a Spark DataFrame from an R DataFrame with <code class="language-plaintext highlighter-rouge">createDataFrame(r_df)</code>, when applying an R native function to each partition
via <code class="language-plaintext highlighter-rouge">dapply(...)</code> and when applying an R native function to grouped data via <code class="language-plaintext highlighter-rouge">gapply(...)</code>.
To use Arrow when executing these, users need to set the Spark configuration ‘spark.sql.execution.arrow.sparkr.enabled’
to ‘true’ first. This is disabled by default.</p>
<p>Whether the optimization is enabled or not, SparkR produces the same results. In addition, the conversion
between Spark DataFrame and R DataFrame falls back automatically to non-Arrow optimization implementation
when the optimization fails for any reasons before the actual computation.</p>
<div data-lang="r">
<figure class="highlight"><pre><code class="language-r" data-lang="r"><span class="c1"># Start up spark session with Arrow optimization enabled</span><span class="w">
</span><span class="n">sparkR.session</span><span class="p">(</span><span class="n">master</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"local[*]"</span><span class="p">,</span><span class="w">
</span><span class="n">sparkConfig</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="nf">list</span><span class="p">(</span><span class="n">spark.sql.execution.arrow.sparkr.enabled</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="s2">"true"</span><span class="p">))</span><span class="w">
</span><span class="c1"># Converts Spark DataFrame from an R DataFrame</span><span class="w">
</span><span class="n">spark_df</span><span class="w"> </span><span class="o">&lt;-</span><span class="w"> </span><span class="n">createDataFrame</span><span class="p">(</span><span class="n">mtcars</span><span class="p">)</span><span class="w">
</span><span class="c1"># Converts Spark DataFrame to an R DataFrame</span><span class="w">
</span><span class="n">collect</span><span class="p">(</span><span class="n">spark_df</span><span class="p">)</span><span class="w">
</span><span class="c1"># Apply an R native function to each partition.</span><span class="w">
</span><span class="n">collect</span><span class="p">(</span><span class="n">dapply</span><span class="p">(</span><span class="n">spark_df</span><span class="p">,</span><span class="w"> </span><span class="k">function</span><span class="p">(</span><span class="n">rdf</span><span class="p">)</span><span class="w"> </span><span class="p">{</span><span class="w"> </span><span class="n">data.frame</span><span class="p">(</span><span class="n">rdf</span><span class="o">$</span><span class="n">gear</span><span class="w"> </span><span class="o">+</span><span class="w"> </span><span class="m">1</span><span class="p">)</span><span class="w"> </span><span class="p">},</span><span class="w"> </span><span class="n">structType</span><span class="p">(</span><span class="s2">"gear double"</span><span class="p">)))</span><span class="w">
</span><span class="c1"># Apply an R native function to grouped data.</span><span class="w">
</span><span class="n">collect</span><span class="p">(</span><span class="n">gapply</span><span class="p">(</span><span class="n">spark_df</span><span class="p">,</span><span class="w">
</span><span class="s2">"gear"</span><span class="p">,</span><span class="w">
</span><span class="k">function</span><span class="p">(</span><span class="n">key</span><span class="p">,</span><span class="w"> </span><span class="n">group</span><span class="p">)</span><span class="w"> </span><span class="p">{</span><span class="w">
</span><span class="n">data.frame</span><span class="p">(</span><span class="n">gear</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">key</span><span class="p">[[</span><span class="m">1</span><span class="p">]],</span><span class="w"> </span><span class="n">disp</span><span class="w"> </span><span class="o">=</span><span class="w"> </span><span class="n">mean</span><span class="p">(</span><span class="n">group</span><span class="o">$</span><span class="n">disp</span><span class="p">)</span><span class="w"> </span><span class="o">&gt;</span><span class="w"> </span><span class="n">group</span><span class="o">$</span><span class="n">disp</span><span class="p">)</span><span class="w">
</span><span class="p">},</span><span class="w">
</span><span class="n">structType</span><span class="p">(</span><span class="s2">"gear double, disp boolean"</span><span class="p">)))</span></code></pre></figure>
</div>
<p>Note that even with Arrow, <code class="language-plaintext highlighter-rouge">collect(spark_df)</code> results in the collection of all records in the DataFrame to
the driver program and should be done on a small subset of the data. In addition, the specified output schema
in <code class="language-plaintext highlighter-rouge">gapply(...)</code> and <code class="language-plaintext highlighter-rouge">dapply(...)</code> should be matched to the R DataFrame&#8217;s returned by the given function.</p>
<h2 id="supported-sql-types">Supported SQL Types</h2>
<p>Currently, all Spark SQL data types are supported by Arrow-based conversion except <code class="language-plaintext highlighter-rouge">FloatType</code>, <code class="language-plaintext highlighter-rouge">BinaryType</code>, <code class="language-plaintext highlighter-rouge">ArrayType</code>, <code class="language-plaintext highlighter-rouge">StructType</code> and <code class="language-plaintext highlighter-rouge">MapType</code>.</p>
<h1 id="r-function-name-conflicts">R Function Name Conflicts</h1>
<p>When loading and attaching a new package in R, it is possible to have a name <a href="https://stat.ethz.ch/R-manual/R-devel/library/base/html/library.html">conflict</a>, where a
function is masking another function.</p>
<p>The following functions are masked by the SparkR package:</p>
<table class="table">
<tr><th>Masked function</th><th>How to Access</th></tr>
<tr>
<td><code>cov</code> in <code>package:stats</code></td>
<td><code><pre>stats::cov(x, y = NULL, use = "everything",
method = c("pearson", "kendall", "spearman"))</pre></code></td>
</tr>
<tr>
<td><code>filter</code> in <code>package:stats</code></td>
<td><code><pre>stats::filter(x, filter, method = c("convolution", "recursive"),
sides = 2, circular = FALSE, init)</pre></code></td>
</tr>
<tr>
<td><code>sample</code> in <code>package:base</code></td>
<td><code>base::sample(x, size, replace = FALSE, prob = NULL)</code></td>
</tr>
</table>
<p>Since part of SparkR is modeled on the <code class="language-plaintext highlighter-rouge">dplyr</code> package, certain functions in SparkR share the same names with those in <code class="language-plaintext highlighter-rouge">dplyr</code>. Depending on the load order of the two packages, some functions from the package loaded first are masked by those in the package loaded after. In such case, prefix such calls with the package name, for instance, <code class="language-plaintext highlighter-rouge">SparkR::cume_dist(x)</code> or <code class="language-plaintext highlighter-rouge">dplyr::cume_dist(x)</code>.</p>
<p>You can inspect the search path in R with <a href="https://stat.ethz.ch/R-manual/R-devel/library/base/html/search.html"><code class="language-plaintext highlighter-rouge">search()</code></a></p>
<h1 id="migration-guide">Migration Guide</h1>
<p>The migration guide is now archived <a href="sparkr-migration-guide.html">on this page</a>.</p>
</div>
<!-- /container -->
</div>
<script src="js/vendor/jquery-3.5.1.min.js"></script>
<script src="js/vendor/bootstrap.bundle.min.js"></script>
<script src="js/vendor/anchor.min.js"></script>
<script src="js/main.js"></script>
<script type="text/javascript" src="https://cdn.jsdelivr.net/npm/docsearch.js@2/dist/cdn/docsearch.min.js"></script>
<script type="text/javascript">
// DocSearch is entirely free and automated. DocSearch is built in two parts:
// 1. a crawler which we run on our own infrastructure every 24 hours. It follows every link
// in your website and extract content from every page it traverses. It then pushes this
// content to an Algolia index.
// 2. a JavaScript snippet to be inserted in your website that will bind this Algolia index
// to your search input and display its results in a dropdown UI. If you want to find more
// details on how works DocSearch, check the docs of DocSearch.
docsearch({
apiKey: 'b18ca3732c502995563043aa17bc6ecb',
indexName: 'apache_spark',
inputSelector: '#docsearch-input',
enhancedSearchInput: true,
algoliaOptions: {
'facetFilters': ["version:3.2.0"]
},
debug: false // Set debug to true if you want to inspect the dropdown
});
</script>
<!-- MathJax Section -->
<script type="text/x-mathjax-config">
MathJax.Hub.Config({
TeX: { equationNumbers: { autoNumber: "AMS" } }
});
</script>
<script>
// Note that we load MathJax this way to work with local file (file://), HTTP and HTTPS.
// We could use "//cdn.mathjax...", but that won't support "file://".
(function(d, script) {
script = d.createElement('script');
script.type = 'text/javascript';
script.async = true;
script.onload = function(){
MathJax.Hub.Config({
tex2jax: {
inlineMath: [ ["$", "$"], ["\\\\(","\\\\)"] ],
displayMath: [ ["$$","$$"], ["\\[", "\\]"] ],
processEscapes: true,
skipTags: ['script', 'noscript', 'style', 'textarea', 'pre']
}
});
};
script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') +
'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' +
'?config=TeX-AMS-MML_HTMLorMML';
d.getElementsByTagName('head')[0].appendChild(script);
}(document));
</script>
</body>
</html>